You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2018/04/10 08:59:29 UTC

[01/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI

Repository: ignite
Updated Branches:
  refs/heads/master a0a187bf0 -> a64b941df


http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/zookeeper/ZkTestClientCnxnSocketNIO.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/zookeeper/ZkTestClientCnxnSocketNIO.java b/modules/zookeeper/src/test/java/org/apache/zookeeper/ZkTestClientCnxnSocketNIO.java
new file mode 100644
index 0000000..7892b5e
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/zookeeper/ZkTestClientCnxnSocketNIO.java
@@ -0,0 +1,137 @@
+/*
+ * 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.zookeeper;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.channels.SelectionKey;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.logger.java.JavaLogger;
+import org.apache.ignite.testframework.GridTestUtils;
+
+/**
+ *
+ */
+public class ZkTestClientCnxnSocketNIO extends ClientCnxnSocketNIO {
+    /** */
+    public static final IgniteLogger log = new JavaLogger().getLogger(ZkTestClientCnxnSocketNIO.class);
+
+    /** */
+    public static volatile boolean DEBUG = false;
+
+    /** */
+    public volatile CountDownLatch blockConnectLatch;
+
+    /** */
+    public static ConcurrentHashMap<String, ZkTestClientCnxnSocketNIO> clients = new ConcurrentHashMap<>();
+
+    /** */
+    private final String nodeName;
+
+    /**
+     *
+     */
+    public static void reset() {
+        clients.clear();
+    }
+
+    /**
+     * @param node Node.
+     * @return ZK client.
+     */
+    public static ZkTestClientCnxnSocketNIO forNode(Ignite node) {
+        return clients.get(node.name());
+    }
+
+    /**
+     * @param instanceName Ignite instance name.
+     * @return ZK client.
+     */
+    public static ZkTestClientCnxnSocketNIO forNode(String instanceName) {
+        return clients.get(instanceName);
+    }
+
+    /**
+     * @throws IOException If failed.
+     */
+    public ZkTestClientCnxnSocketNIO() throws IOException {
+        super();
+
+        String threadName = Thread.currentThread().getName();
+
+        nodeName = threadName.substring(threadName.indexOf('-') + 1);
+
+        if (DEBUG)
+            log.info("ZkTestClientCnxnSocketNIO created for node: " + nodeName);
+    }
+
+    /** {@inheritDoc} */
+    @Override void connect(InetSocketAddress addr) throws IOException {
+        CountDownLatch blockConnect = this.blockConnectLatch;
+
+        if (DEBUG)
+            log.info("ZkTestClientCnxnSocketNIO connect [node=" + nodeName + ", addr=" + addr + ']');
+
+        if (blockConnect != null && blockConnect.getCount() > 0) {
+            try {
+                log.info("ZkTestClientCnxnSocketNIO block connect");
+
+                blockConnect.await(60, TimeUnit.SECONDS);
+
+                log.info("ZkTestClientCnxnSocketNIO finish block connect");
+            }
+            catch (Exception e) {
+                log.error("Error in ZkTestClientCnxnSocketNIO: " + e, e);
+            }
+        }
+
+        super.connect(addr);
+
+        clients.put(nodeName, this);
+    }
+
+    /**
+     *
+     */
+    public void allowConnect() {
+        assert blockConnectLatch != null && blockConnectLatch.getCount() == 1 : blockConnectLatch;
+
+        log.info("ZkTestClientCnxnSocketNIO allowConnect [node=" + nodeName + ']');
+
+        blockConnectLatch.countDown();
+    }
+
+    /**
+     * @param blockConnect {@code True} to block client reconnect.
+     * @throws Exception If failed.
+     */
+    public void closeSocket(boolean blockConnect) throws Exception {
+        if (blockConnect)
+            blockConnectLatch = new CountDownLatch(1);
+
+        log.info("ZkTestClientCnxnSocketNIO closeSocket [node=" + nodeName + ", block=" + blockConnect + ']');
+
+        SelectionKey k = GridTestUtils.getFieldValue(this, ClientCnxnSocketNIO.class, "sockKey");
+
+        k.channel().close();
+    }
+}


[12/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI

Posted by ag...@apache.org.
IGNITE-7222 Added ZooKeeper discovery SPI


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

Branch: refs/heads/master
Commit: a64b941df0a4d3bfc3a2dab32f85c371c1a509be
Parents: a0a187b
Author: Semyon Boikov <sb...@apache.org>
Authored: Tue Apr 10 11:37:39 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Apr 10 11:57:40 2018 +0300

----------------------------------------------------------------------
 .../jdbc2/JdbcAbstractDmlStatementSelfTest.java |    6 +-
 .../CommunicationFailureContext.java            |   62 +
 .../CommunicationFailureResolver.java           |   28 +
 .../DefaultCommunicationFailureResolver.java    |  305 ++
 .../configuration/IgniteConfiguration.java      |   22 +
 .../org/apache/ignite/internal/GridTopic.java   |    3 +
 .../apache/ignite/internal/IgniteKernal.java    |    5 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |    3 +
 .../internal/managers/GridManagerAdapter.java   |    8 +
 .../managers/communication/GridIoManager.java   |    4 +-
 .../communication/GridIoMessageFactory.java     |   12 +
 .../discovery/CustomMessageWrapper.java         |    5 +
 .../internal/managers/discovery/DiscoCache.java |    8 +
 .../discovery/DiscoveryCustomMessage.java       |   10 +-
 .../DiscoveryMessageResultsCollector.java       |  222 +
 .../discovery/GridDiscoveryManager.java         |  128 +-
 .../managers/discovery/IgniteClusterNode.java   |   69 +
 .../managers/discovery/IgniteDiscoverySpi.java  |   67 +
 .../IgniteDiscoverySpiInternalListener.java     |   42 +
 .../authentication/UserAcceptedMessage.java     |    5 +
 .../authentication/UserProposedMessage.java     |    5 +
 .../cache/CacheAffinityChangeMessage.java       |    5 +
 .../cache/CacheAffinitySharedManager.java       |   26 +-
 .../cache/CacheStatisticsModeChangeMessage.java |    5 +
 .../ClientCacheChangeDiscoveryMessage.java      |    5 +
 .../ClientCacheChangeDummyDiscoveryMessage.java |    5 +
 .../cache/DynamicCacheChangeBatch.java          |    5 +
 .../processors/cache/GridCacheAdapter.java      |    3 +-
 .../GridCachePartitionExchangeManager.java      |   16 +-
 .../processors/cache/GridCacheProcessor.java    |    4 +-
 .../processors/cache/GridCacheUtils.java        |    6 +-
 .../processors/cache/WalStateFinishMessage.java |    5 +
 .../cache/WalStateProposeMessage.java           |    5 +
 .../cache/binary/BinaryMetadataTransport.java   |   24 +-
 .../binary/MetadataUpdateAcceptedMessage.java   |    5 +
 .../binary/MetadataUpdateProposedMessage.java   |    5 +
 .../dht/GridClientPartitionTopology.java        |   39 +
 .../distributed/dht/GridDhtCacheAdapter.java    |    4 +
 .../dht/GridDhtPartitionTopology.java           |    6 +
 .../dht/GridDhtPartitionTopologyImpl.java       |   39 +
 .../GridDhtPartitionsExchangeFuture.java        |   26 +-
 .../cluster/ChangeGlobalStateFinishMessage.java |    5 +
 .../cluster/ChangeGlobalStateMessage.java       |    5 +
 .../cluster/ClusterMetricsUpdateMessage.java    |  158 +
 .../processors/cluster/ClusterNodeMetrics.java  |   62 +
 .../processors/cluster/ClusterProcessor.java    |  249 +-
 .../continuous/AbstractContinuousMessage.java   |    5 +
 .../continuous/ContinuousRoutineInfo.java       |  100 +
 .../ContinuousRoutineStartResultMessage.java    |  206 +
 .../ContinuousRoutinesCommonDiscoveryData.java  |   45 +
 .../continuous/ContinuousRoutinesInfo.java      |  132 +
 ...tinuousRoutinesJoiningNodeDiscoveryData.java |   45 +
 .../continuous/GridContinuousProcessor.java     |  862 +++-
 .../continuous/StartRequestDataV2.java          |  164 +
 .../StartRoutineDiscoveryMessageV2.java         |   77 +
 .../StopRoutineAckDiscoveryMessage.java         |    5 +
 .../datastreamer/DataStreamerImpl.java          |   27 +-
 .../marshaller/MappingAcceptedMessage.java      |    5 +
 .../marshaller/MappingProposedMessage.java      |    5 +
 .../message/SchemaFinishDiscoveryMessage.java   |    5 +
 .../message/SchemaProposeDiscoveryMessage.java  |    5 +
 .../ignite/internal/util/nio/GridNioServer.java |   18 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   10 +
 .../org/apache/ignite/spi/IgniteSpiContext.java |   11 +
 .../communication/tcp/TcpCommunicationSpi.java  |  190 +-
 .../tcp/internal/ConnectionKey.java             |  117 +
 .../TcpCommunicationConnectionCheckFuture.java  |  519 ++
 ...pCommunicationNodeConnectionCheckFuture.java |   30 +
 .../discovery/DiscoverySpiCustomMessage.java    |   15 +-
 ...DiscoverySpiMutableCustomMessageSupport.java |   40 +
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   55 +-
 .../tcp/internal/TcpDiscoveryNode.java          |   32 +-
 .../resources/META-INF/classnames.properties    |    2 +
 ...unctionExcludeNeighborsAbstractSelfTest.java |    8 +-
 .../failure/FailureHandlerTriggeredTest.java    |    4 +
 .../internal/ClusterGroupHostsSelfTest.java     |    3 +
 .../ignite/internal/ClusterGroupSelfTest.java   |    2 +
 .../internal/ClusterNodeMetricsUpdateTest.java  |  173 +
 .../internal/DiscoverySpiTestListener.java      |  162 +
 .../ignite/internal/GridDiscoverySelfTest.java  |   14 +-
 .../GridJobMasterLeaveAwareSelfTest.java        |    2 +
 .../internal/GridJobStealingSelfTest.java       |    2 +
 .../internal/GridSameVmStartupSelfTest.java     |   19 +-
 .../apache/ignite/internal/GridSelfTest.java    |    2 +
 .../IgniteClientReconnectAbstractTest.java      |   53 +-
 .../IgniteClientReconnectApiExceptionTest.java  |   21 +-
 .../IgniteClientReconnectAtomicsTest.java       |   30 +-
 .../IgniteClientReconnectCacheTest.java         |   49 +-
 .../IgniteClientReconnectCollectionsTest.java   |   14 +-
 .../IgniteClientReconnectComputeTest.java       |    6 +-
 ...eClientReconnectContinuousProcessorTest.java |   13 +-
 ...IgniteClientReconnectDiscoveryStateTest.java |   22 +-
 ...niteClientReconnectFailoverAbstractTest.java |   12 +-
 .../IgniteClientReconnectServicesTest.java      |    8 +-
 .../internal/IgniteClientReconnectStopTest.java |   12 +-
 .../IgniteClientReconnectStreamerTest.java      |    4 +-
 .../ignite/internal/IgniteClientRejoinTest.java |    3 +
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   16 +-
 .../GridAffinityProcessorAbstractSelfTest.java  |    4 +-
 .../CacheMetricsForClusterGroupSelfTest.java    |   12 +-
 .../cache/GridCacheAbstractSelfTest.java        |    2 +
 .../cache/IgniteCacheNearLockValueSelfTest.java |    4 +-
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |   11 +
 .../IgniteClusterActivateDeactivateTest.java    |   65 +
 .../IgniteDaemonNodeMarshallerCacheTest.java    |    3 +-
 .../binary/BinaryMetadataUpdatesFlowTest.java   |   12 +-
 ...ntNodeBinaryObjectMetadataMultinodeTest.java |    2 +-
 .../GridCacheQueueClientDisconnectTest.java     |   10 +
 .../IgniteClientDataStructuresAbstractTest.java |    3 +-
 .../CacheLateAffinityAssignmentTest.java        |  127 +-
 .../GridCacheNodeFailureAbstractTest.java       |    5 +-
 .../distributed/IgniteCache150ClientsTest.java  |    2 +
 .../distributed/IgniteCacheManyClientsTest.java |   44 +-
 .../IgniteOptimisticTxSuspendResumeTest.java    |    2 +
 ...ridCacheDhtPreloadMultiThreadedSelfTest.java |    4 +
 .../dht/GridCacheDhtPreloadSelfTest.java        |    2 +
 .../dht/TxRecoveryStoreEnabledTest.java         |   15 +-
 ...titionedExplicitLockNodeFailureSelfTest.java |    3 +-
 .../ClientReconnectContinuousQueryTest.java     |   19 +-
 ...yRemoteFilterMissingInClassPathSelfTest.java |   23 +-
 ...CacheContinuousQueryClientReconnectTest.java |    3 +
 .../CacheVersionedEntryAbstractTest.java        |   33 +-
 .../continuous/GridEventConsumeSelfTest.java    |   34 +-
 .../service/ClosureServiceClientsNodesTest.java |   19 +-
 .../internal/util/GridTestClockTimer.java       |    9 +
 .../GridMarshallerMappingConsistencyTest.java   |    4 +
 .../ignite/messaging/GridMessagingSelfTest.java |  126 +-
 .../GridTcpCommunicationSpiAbstractTest.java    |   71 +
 .../FilterDataForClientNodeDiscoveryTest.java   |    5 +
 .../testframework/GridSpiTestContext.java       |   10 +
 .../config/GridTestProperties.java              |    9 +
 .../testframework/junits/GridAbstractTest.java  |  129 +-
 .../junits/multijvm/IgniteNodeRunner.java       |    2 +
 .../testsuites/IgniteComputeGridTestSuite.java  |    2 +
 ...niteCacheDistributedQueryCancelSelfTest.java |    2 +-
 .../DynamicIndexAbstractBasicSelfTest.java      |    5 +-
 .../GridJtaTransactionManagerSelfTest.java      |   21 +-
 .../GridPartitionedCacheJtaFactorySelfTest.java |   19 +-
 .../org/apache/ignite/spark/IgniteRDD.scala     |    9 +-
 .../ignite/internal/GridFactorySelfTest.java    |    3 +-
 .../p2p/GridP2PUserVersionChangeSelfTest.java   |    5 +-
 modules/yardstick/pom-standalone.xml            |    6 +
 modules/yardstick/pom.xml                       |    6 +
 modules/zookeeper/pom.xml                       |   40 +
 .../spi/discovery/zk/ZookeeperDiscoverySpi.java |  557 ++
 .../zk/internal/ZkAbstractCallabck.java         |   83 +
 .../zk/internal/ZkAbstractChildrenCallback.java |   61 +
 .../zk/internal/ZkAbstractWatcher.java          |   55 +
 .../discovery/zk/internal/ZkAliveNodeData.java  |   40 +
 .../zk/internal/ZkBulkJoinContext.java          |   50 +
 .../discovery/zk/internal/ZkClusterNodes.java   |  103 +
 .../internal/ZkCommunicationErrorNodeState.java |   46 +
 .../ZkCommunicationErrorProcessFuture.java      |  411 ++
 ...kCommunicationErrorResolveFinishMessage.java |   69 +
 .../ZkCommunicationErrorResolveResult.java      |   45 +
 ...ZkCommunicationErrorResolveStartMessage.java |   61 +
 .../internal/ZkCommunicationFailureContext.java |  188 +
 .../zk/internal/ZkDiscoveryCustomEventData.java |   89 +
 .../zk/internal/ZkDiscoveryEventData.java       |  165 +
 .../zk/internal/ZkDiscoveryEventsData.java      |  121 +
 .../internal/ZkDiscoveryNodeFailEventData.java  |   55 +
 .../internal/ZkDiscoveryNodeJoinEventData.java  |   60 +
 .../ZkDistributedCollectDataFuture.java         |  250 +
 .../zk/internal/ZkForceNodeFailMessage.java     |   65 +
 .../discovery/zk/internal/ZkIgnitePaths.java    |  307 ++
 .../zk/internal/ZkInternalJoinErrorMessage.java |   44 +
 .../zk/internal/ZkInternalMessage.java          |   27 +
 .../zk/internal/ZkJoinEventDataForJoined.java   |   83 +
 .../zk/internal/ZkJoinedNodeEvtData.java        |   79 +
 .../zk/internal/ZkJoiningNodeData.java          |   87 +
 .../zk/internal/ZkNoServersMessage.java         |   50 +
 .../zk/internal/ZkNodeValidateResult.java       |   43 +
 .../spi/discovery/zk/internal/ZkRunnable.java   |   51 +
 .../discovery/zk/internal/ZkRuntimeState.java   |  135 +
 .../discovery/zk/internal/ZkTimeoutObject.java  |   54 +
 .../discovery/zk/internal/ZookeeperClient.java  | 1219 +++++
 .../ZookeeperClientFailedException.java         |   40 +
 .../zk/internal/ZookeeperClusterNode.java       |  362 ++
 .../zk/internal/ZookeeperDiscoveryImpl.java     | 4464 ++++++++++++++++
 .../java/org/apache/ZookeeperNodeStart.java     |   46 +
 ...CacheEntryListenerWithZkDiscoAtomicTest.java |   32 +
 .../ZookeeperDiscoverySpiAbstractTestSuite.java |  118 +
 .../zk/ZookeeperDiscoverySpiTestSuite1.java     |   44 +
 .../zk/ZookeeperDiscoverySpiTestSuite2.java     |   94 +
 ...ZookeeperDiscoverySuitePreprocessorTest.java |  101 +
 .../zk/internal/ZookeeperClientTest.java        |  495 ++
 ...okeeperDiscoverySpiSaslAuthAbstractTest.java |  247 +
 ...ZookeeperDiscoverySpiSaslFailedAuthTest.java |   44 +
 ...eeperDiscoverySpiSaslSuccessfulAuthTest.java |   48 +
 .../zk/internal/ZookeeperDiscoverySpiTest.java  | 4847 ++++++++++++++++++
 .../zookeeper/ZkTestClientCnxnSocketNIO.java    |  137 +
 191 files changed, 21158 insertions(+), 777 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
index f4c0ca3..0a055a9 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
@@ -138,8 +138,10 @@ public abstract class JdbcAbstractDmlStatementSelfTest extends GridCommonAbstrac
     @Override protected void afterTest() throws Exception {
         ((IgniteEx)ignite(0)).context().cache().dynamicDestroyCache(DEFAULT_CACHE_NAME, true, true, false);
 
-        conn.close();
-        assertTrue(conn.isClosed());
+        if (conn != null) {
+            conn.close();
+            assertTrue(conn.isClosed());
+        }
 
         cleanUpWorkingDir();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureContext.java b/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureContext.java
new file mode 100644
index 0000000..a32d38c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureContext.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.configuration;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.spi.communication.CommunicationSpi;
+
+/**
+ * Communication Failure Context.
+ */
+public interface CommunicationFailureContext {
+    /**
+     * @return Current topology snapshot.
+     */
+    public List<ClusterNode> topologySnapshot();
+
+    /**
+     * @param node1 First node.
+     * @param node2 Second node.
+     * @return {@code True} if {@link CommunicationSpi} is able to establish connection from first node to second node.
+     */
+    public boolean connectionAvailable(ClusterNode node1, ClusterNode node2);
+
+    /**
+     * @return Currently started caches.
+     */
+    public Map<String, CacheConfiguration<?, ?>> startedCaches();
+
+    /**
+     * @param cacheName Cache name.
+     * @return Cache partitions affinity assignment.
+     */
+    public List<List<ClusterNode>> cacheAffinity(String cacheName);
+
+    /**
+     * @param cacheName Cache name.
+     * @return Cache partitions owners.
+     */
+    public List<List<ClusterNode>> cachePartitionOwners(String cacheName);
+
+    /**
+     * @param node Node to kill.
+     */
+    public void killNode(ClusterNode node);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureResolver.java b/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureResolver.java
new file mode 100644
index 0000000..a4d92f3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureResolver.java
@@ -0,0 +1,28 @@
+/*
+ * 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.configuration;
+
+/**
+ * Communication Failure Resolver.
+ */
+public interface CommunicationFailureResolver {
+    /**
+     * @param ctx Context.
+     */
+    public void resolve(CommunicationFailureContext ctx);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/configuration/DefaultCommunicationFailureResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/DefaultCommunicationFailureResolver.java b/modules/core/src/main/java/org/apache/ignite/configuration/DefaultCommunicationFailureResolver.java
new file mode 100644
index 0000000..a4c6da9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/DefaultCommunicationFailureResolver.java
@@ -0,0 +1,305 @@
+/*
+ * 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.configuration;
+
+import java.util.BitSet;
+import java.util.List;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+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;
+import org.apache.ignite.resources.LoggerResource;
+
+/**
+ * Default Communication Failure Resolver.
+ */
+public class DefaultCommunicationFailureResolver implements CommunicationFailureResolver {
+    /** */
+    @LoggerResource
+    private IgniteLogger log;
+
+    /** {@inheritDoc} */
+    @Override public void resolve(CommunicationFailureContext ctx) {
+        ClusterGraph graph = new ClusterGraph(log, ctx);
+
+        ClusterSearch cluster = graph.findLargestIndependentCluster();
+
+        List<ClusterNode> nodes = ctx.topologySnapshot();
+
+        assert nodes.size() > 0;
+        assert cluster != null;
+
+        if (graph.checkFullyConnected(cluster.nodesBitSet)) {
+            assert cluster.nodeCnt <= nodes.size();
+
+            if (cluster.nodeCnt < nodes.size()) {
+                if (log.isInfoEnabled()) {
+                    log.info("Communication problem resolver found fully connected independent cluster [" +
+                        "clusterSrvCnt=" + cluster.srvCnt +
+                        ", clusterTotalNodes=" + cluster.nodeCnt +
+                        ", totalAliveNodes=" + nodes.size() + "]");
+                }
+
+                for (int i = 0; i < nodes.size(); i++) {
+                    if (!cluster.nodesBitSet.get(i))
+                        ctx.killNode(nodes.get(i));
+                }
+            }
+            else
+                U.warn(log, "All alive nodes are fully connected, this should be resolved automatically.");
+        }
+        else {
+            if (log.isInfoEnabled()) {
+                log.info("Communication problem resolver failed to find fully connected independent cluster.");
+            }
+        }
+    }
+
+    /**
+     * @param cluster Cluster nodes mask.
+     * @param nodes Nodes.
+     * @param limit IDs limit.
+     * @return Cluster node IDs string.
+     */
+    private static String clusterNodeIds(BitSet cluster, List<ClusterNode> nodes, int limit) {
+        int startIdx = 0;
+
+        StringBuilder builder = new StringBuilder();
+
+        int cnt = 0;
+
+        for (;;) {
+            int idx = cluster.nextSetBit(startIdx);
+
+            if (idx == -1)
+                break;
+
+            startIdx = idx + 1;
+
+            if (builder.length() == 0) {
+                builder.append('[');
+            }
+            else
+                builder.append(", ");
+
+            builder.append(nodes.get(idx).id());
+
+            if (cnt++ > limit)
+                builder.append(", ...");
+        }
+
+        builder.append(']');
+
+        return builder.toString();
+    }
+
+    /**
+     *
+     */
+    private static class ClusterSearch {
+        /** */
+        int srvCnt;
+
+        /** */
+        int nodeCnt;
+
+        /** */
+        final BitSet nodesBitSet;
+
+        /**
+         * @param nodes Total nodes.
+         */
+        ClusterSearch(int nodes) {
+            nodesBitSet = new BitSet(nodes);
+        }
+    }
+
+    /**
+     *
+     */
+    private static class ClusterGraph {
+        /** */
+        private final static int WORD_IDX_SHIFT = 6;
+
+        /** */
+        private final IgniteLogger log;
+
+        /** */
+        private final int nodeCnt;
+
+        /** */
+        private final long[] visitBitSet;
+
+        /** */
+        private final CommunicationFailureContext ctx;
+
+        /** */
+        private final List<ClusterNode> nodes;
+
+        /**
+         * @param log Logger.
+         * @param ctx Context.
+         */
+        ClusterGraph(IgniteLogger log, CommunicationFailureContext ctx) {
+            this.log = log;
+            this.ctx = ctx;
+
+            nodes = ctx.topologySnapshot();
+
+            nodeCnt = nodes.size();
+
+            assert nodeCnt > 0;
+
+            visitBitSet = initBitSet(nodeCnt);
+        }
+
+        /**
+         * @param bitIndex Bit index.
+         * @return Word index containing bit with given index.
+         */
+        private static int wordIndex(int bitIndex) {
+            return bitIndex >> WORD_IDX_SHIFT;
+        }
+
+        /**
+         * @param bitCnt Number of bits.
+         * @return Bit set words.
+         */
+        static long[] initBitSet(int bitCnt) {
+            return new long[wordIndex(bitCnt - 1) + 1];
+        }
+
+        /**
+         * @return Cluster nodes bit set.
+         */
+        ClusterSearch findLargestIndependentCluster() {
+            ClusterSearch maxCluster = null;
+
+            for (int i = 0; i < nodeCnt; i++) {
+                if (getBit(visitBitSet, i))
+                    continue;
+
+                ClusterSearch cluster = new ClusterSearch(nodeCnt);
+
+                search(cluster, i);
+
+                if (log.isInfoEnabled()) {
+                    log.info("Communication problem resolver found cluster [srvCnt=" + cluster.srvCnt +
+                        ", totalNodeCnt=" + cluster.nodeCnt +
+                        ", nodeIds=" + clusterNodeIds(cluster.nodesBitSet, nodes, 1000) + "]");
+                }
+
+                if (maxCluster == null || cluster.srvCnt > maxCluster.srvCnt)
+                    maxCluster = cluster;
+            }
+
+            return maxCluster;
+        }
+
+        /**
+         * @param cluster Cluster nodes bit set.
+         * @return {@code True} if all cluster nodes are able to connect to each other.
+         */
+        boolean checkFullyConnected(BitSet cluster) {
+            int startIdx = 0;
+
+            int clusterNodes = cluster.cardinality();
+
+            for (;;) {
+                int idx = cluster.nextSetBit(startIdx);
+
+                if (idx == -1)
+                    break;
+
+                ClusterNode node1 = nodes.get(idx);
+
+                for (int i = 0; i < clusterNodes; i++) {
+                    if (!cluster.get(i) || i == idx)
+                        continue;
+
+                    ClusterNode node2 = nodes.get(i);
+
+                    if (cluster.get(i) && !ctx.connectionAvailable(node1, node2))
+                        return false;
+                }
+
+                startIdx = idx + 1;
+            }
+
+            return true;
+        }
+
+        /**
+         * @param cluster Current cluster bit set.
+         * @param idx Node index.
+         */
+        void search(ClusterSearch cluster, int idx) {
+            assert !getBit(visitBitSet, idx);
+
+            setBit(visitBitSet, idx);
+
+            cluster.nodesBitSet.set(idx);
+            cluster.nodeCnt++;
+
+            ClusterNode node1 = nodes.get(idx);
+
+            if (!CU.clientNode(node1))
+                cluster.srvCnt++;
+
+            for (int i = 0; i < nodeCnt; i++) {
+                if (i == idx || getBit(visitBitSet, i))
+                    continue;
+
+                ClusterNode node2 = nodes.get(i);
+
+                boolean connected = ctx.connectionAvailable(node1, node2) ||
+                    ctx.connectionAvailable(node2, node1);
+
+                if (connected)
+                    search(cluster, i);
+            }
+        }
+
+        /**
+         * @param words Bit set words.
+         * @param bitIndex Bit index.
+         */
+        static void setBit(long words[], int bitIndex) {
+            int wordIndex = wordIndex(bitIndex);
+
+            words[wordIndex] |= (1L << bitIndex);
+        }
+
+        /**
+         * @param words Bit set words.
+         * @param bitIndex Bit index.
+         * @return Bit value.
+         */
+        static boolean getBit(long[] words, int bitIndex) {
+            int wordIndex = wordIndex(bitIndex);
+
+            return (words[wordIndex] & (1L << bitIndex)) != 0;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(DefaultCommunicationFailureResolver.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index add3880..cc3ea10 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -493,6 +493,9 @@ public class IgniteConfiguration {
     /** Failure handler. */
     private FailureHandler failureHnd;
 
+    /** Communication failure resolver */
+    private CommunicationFailureResolver commFailureRslvr;
+
     /**
      * Creates valid grid configuration with all default values.
      */
@@ -520,6 +523,8 @@ public class IgniteConfiguration {
         loadBalancingSpi = cfg.getLoadBalancingSpi();
         indexingSpi = cfg.getIndexingSpi();
 
+        commFailureRslvr = cfg.getCommunicationFailureResolver();
+
         /*
          * Order alphabetically for maintenance purposes.
          */
@@ -607,6 +612,23 @@ public class IgniteConfiguration {
     }
 
     /**
+     * @return Communication failure resovler.
+     */
+    public CommunicationFailureResolver getCommunicationFailureResolver() {
+        return commFailureRslvr;
+    }
+
+    /**
+     * @param commFailureRslvr Communication failure resovler.
+     * @return {@code this} instance.
+     */
+    public IgniteConfiguration setCommunicationFailureResolver(CommunicationFailureResolver commFailureRslvr) {
+        this.commFailureRslvr = commFailureRslvr;
+
+        return this;
+    }
+
+    /**
      * Gets optional grid name. Returns {@code null} if non-default grid name was not
      * provided.
      * <p>The name only works locally and has no effect on topology</p>

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java b/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
index 4932e67..1227e8c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
@@ -121,6 +121,9 @@ public enum GridTopic {
     TOPIC_WAL,
 
     /** */
+    TOPIC_METRICS,
+
+    /** */
     TOPIC_AUTH;
 
     /** Enum values. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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 8bc46fd..0b102e5 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
@@ -1298,7 +1298,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         ackStart(rtBean);
 
         if (!isDaemon())
-            ctx.discovery().ackTopology(localNode().order());
+            ctx.discovery().ackTopology(ctx.discovery().localJoin().joinTopologyVersion().topologyVersion());
     }
 
     /**
@@ -2623,6 +2623,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         objs.add(cfg.getGridLogger());
         objs.add(cfg.getMBeanServer());
 
+        if (cfg.getCommunicationFailureResolver() != null)
+            objs.add(cfg.getCommunicationFailureResolver());
+
         return objs;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 4708dd3..417ba1e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -75,6 +75,7 @@ import org.apache.ignite.failure.FailureContext;
 import org.apache.ignite.failure.FailureType;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
 import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor;
 import org.apache.ignite.internal.processors.igfs.IgfsThreadFactory;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
@@ -2243,6 +2244,8 @@ public class IgnitionEx {
 
             initializeDefaultSpi(myCfg);
 
+            GridDiscoveryManager.initCommunicationErrorResolveConfiguration(myCfg);
+
             initializeDefaultCacheConfiguration(myCfg);
 
             ExecutorConfiguration[] execCfgs = myCfg.getExecutorConfiguration();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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 74f5a10..b0756cf 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
@@ -618,6 +618,14 @@ public abstract class GridManagerAdapter<T extends IgniteSpi> implements GridMan
                         return ctx.nodeAttributes();
                     }
 
+                    @Override public boolean communicationFailureResolveSupported() {
+                        return ctx.discovery().communicationErrorResolveSupported();
+                    }
+
+                    @Override public void resolveCommunicationFailure(ClusterNode node, Exception err) {
+                        ctx.discovery().resolveCommunicationError(node, err);
+                    }
+
                     /**
                      * @param e Exception to handle.
                      * @return GridSpiException Converted exception.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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 d5cdd2d..8d9a700 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
@@ -298,9 +298,9 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
 
                 @Override public MessageReader reader(UUID rmtNodeId, MessageFactory msgFactory)
                     throws IgniteCheckedException {
-                    assert rmtNodeId != null;
 
-                    return new DirectMessageReader(msgFactory, U.directProtocolVersion(ctx, rmtNodeId));
+                    return new DirectMessageReader(msgFactory,
+                        rmtNodeId != null ? U.directProtocolVersion(ctx, rmtNodeId) : GridIoManager.DIRECT_PROTO_VER);
                 }
             };
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index a0fc2f8..5616fd0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@ -123,6 +123,8 @@ import org.apache.ignite.internal.processors.cache.transactions.TxLocksResponse;
 import org.apache.ignite.internal.processors.cache.version.GridCacheRawVersionedEntry;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionEx;
+import org.apache.ignite.internal.processors.cluster.ClusterMetricsUpdateMessage;
+import org.apache.ignite.internal.processors.continuous.ContinuousRoutineStartResultMessage;
 import org.apache.ignite.internal.processors.continuous.GridContinuousMessage;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerEntry;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerRequest;
@@ -909,6 +911,16 @@ public class GridIoMessageFactory implements MessageFactory {
 
                 break;
 
+            case 133:
+                msg = new ClusterMetricsUpdateMessage();
+
+                break;
+
+            case 134:
+                msg = new ContinuousRoutineStartResultMessage();
+
+                break;
+
             // [-3..119] [124..129] [-23..-27] [-36..-55]- this
             // [120..123] - DR
             // [-4..-22, -30..-35] - SQL

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomMessageWrapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomMessageWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomMessageWrapper.java
index 4268886..4b6b7a2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomMessageWrapper.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomMessageWrapper.java
@@ -49,6 +49,11 @@ public class CustomMessageWrapper implements DiscoverySpiCustomMessage {
         return delegate.isMutable();
     }
 
+    /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return delegate.stopProcess();
+    }
+
     /**
      * @return Delegate.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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 c21698f..fef44fa 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
@@ -312,6 +312,14 @@ public class DiscoCache {
     }
 
     /**
+     * @param nodeId Node ID.
+     * @return {@code True} if node is in alives list.
+     */
+    public boolean alive(UUID nodeId) {
+        return alives.contains(nodeId);
+    }
+
+    /**
      * Gets all nodes that have cache with given name.
      *
      * @param cacheName Cache name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java
index c708c62..6ed2096 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.managers.discovery;
 import java.io.Serializable;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddFinishedMessage;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddedMessage;
 import org.jetbrains.annotations.Nullable;
@@ -87,11 +88,18 @@ public interface DiscoveryCustomMessage extends Serializable {
     @Nullable public DiscoveryCustomMessage ackMessage();
 
     /**
-     * @return {@code true} if message can be modified during listener notification. Changes will be send to next nodes.
+     * @return {@code True} if message can be modified during listener notification. Changes will be sent to next nodes.
      */
     public boolean isMutable();
 
     /**
+     * See {@link DiscoverySpiCustomMessage#stopProcess()}.
+     *
+     * @return {@code True} if message should not be sent to others nodes after it was processed on coordinator.
+     */
+    public boolean stopProcess();
+
+    /**
      * Creates new discovery cache if message caused topology version change.
      *
      * @param mgr Discovery manager.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryMessageResultsCollector.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryMessageResultsCollector.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryMessageResultsCollector.java
new file mode 100644
index 0000000..43be952
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryMessageResultsCollector.java
@@ -0,0 +1,222 @@
+/*
+ * 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 java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+public abstract class DiscoveryMessageResultsCollector<M, R>  {
+    /** */
+    private final Map<UUID, NodeMessage<M>> rcvd = new HashMap<>();
+
+    /** */
+    private int leftMsgs;
+
+    /** */
+    protected DiscoCache discoCache;
+
+    /** */
+    protected final GridKernalContext ctx;
+
+    /**
+     * @param ctx Context.
+     */
+    protected DiscoveryMessageResultsCollector(GridKernalContext ctx) {
+        this.ctx = ctx;
+    }
+
+    /**
+     * @param rcvd Received messages.
+     * @return Result.
+     */
+    protected abstract R createResult(Map<UUID, NodeMessage<M>> rcvd);
+
+    /**
+     * @param r Result.
+     */
+    protected abstract void onResultsCollected(R r);
+
+    /**
+     * @param discoCache Discovery state when discovery message was received.
+     * @param node Node.
+     * @return {@code True} if need wait for result from given node.
+     */
+    protected abstract boolean waitForNode(DiscoCache discoCache, ClusterNode node);
+
+    /**
+     * @param discoCache Discovery state.
+     */
+    public final void init(DiscoCache discoCache) {
+        assert discoCache != null;
+
+        R res = null;
+
+        synchronized (this) {
+            assert this.discoCache == null;
+            assert leftMsgs == 0 : leftMsgs;
+
+            this.discoCache = discoCache;
+
+            for (ClusterNode node : discoCache.allNodes()) {
+                if (ctx.discovery().alive(node) && waitForNode(discoCache, node) && !rcvd.containsKey(node.id())) {
+                    rcvd.put(node.id(), new NodeMessage<>((M)null));
+
+                    leftMsgs++;
+                }
+            }
+
+            if (leftMsgs == 0)
+                res = createResult(rcvd);
+        }
+
+        if (res != null)
+            onResultsCollected(res);
+    }
+
+    /**
+     * @param nodeId Node ID.
+     * @param msg Message.
+     */
+    public final void onMessage(UUID nodeId, M msg) {
+        R res = null;
+
+        synchronized (this) {
+            if (allReceived())
+                return;
+
+            NodeMessage<M> expMsg = rcvd.get(nodeId);
+
+            if (expMsg == null)
+                rcvd.put(nodeId, new NodeMessage<>(msg));
+            else if (expMsg.set(msg)) {
+                assert leftMsgs > 0;
+
+                leftMsgs--;
+
+                if (allReceived())
+                    res = createResult(rcvd);
+            }
+        }
+
+        if (res != null)
+            onResultsCollected(res);
+    }
+
+    /**
+     * @param nodeId Failed node ID.
+     */
+    public final void onNodeFail(UUID nodeId) {
+        R res = null;
+
+        synchronized (this) {
+            if (allReceived())
+                return;
+
+            NodeMessage expMsg = rcvd.get(nodeId);
+
+            if (expMsg != null && expMsg.onNodeFailed()) {
+                assert leftMsgs > 0 : leftMsgs;
+
+                leftMsgs--;
+
+                if (allReceived())
+                    res = createResult(rcvd);
+            }
+        }
+
+        if (res != null)
+            onResultsCollected(res);
+    }
+
+    /**
+     * @return {@code True} if expected messages are initialized and all message are received.
+     */
+    private boolean allReceived() {
+        return discoCache != null && leftMsgs == 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(DiscoveryMessageResultsCollector.class, this);
+    }
+
+    /**
+     *
+     */
+    protected static class NodeMessage<M> {
+        /** */
+        boolean nodeFailed;
+
+        /** */
+        M msg;
+
+        /**
+         * @param msg Message.
+         */
+        NodeMessage(M msg) {
+            this.msg = msg;
+        }
+
+        /**
+         * @return Message or {@code null} if node failed.
+         */
+        @Nullable public M message() {
+            return msg;
+        }
+
+        /**
+         * @return {@code True} if node result was not set before.
+         */
+        boolean onNodeFailed() {
+            if (nodeFailed || msg != null)
+                return false;
+
+            nodeFailed = true;
+
+            return true;
+        }
+
+        /**
+         * @param msg Received message.
+         * @return {@code True} if node result was not set before.
+         */
+        boolean set(M msg) {
+            assert msg != null;
+
+            if (this.msg != null)
+                return false;
+
+            this.msg = msg;
+
+            return !nodeFailed;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(NodeMessage.class, this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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 2e814d4..4c5690e 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
@@ -54,8 +54,11 @@ import org.apache.ignite.cluster.BaselineNode;
 import org.apache.ignite.cluster.ClusterMetrics;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.CommunicationFailureResolver;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.DefaultCommunicationFailureResolver;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.failure.FailureContext;
@@ -112,6 +115,8 @@ import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.plugin.security.SecurityCredentials;
 import org.apache.ignite.plugin.segmentation.SegmentationPolicy;
 import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.communication.CommunicationSpi;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag.JoiningNodeDiscoveryData;
 import org.apache.ignite.spi.discovery.DiscoveryMetricsProvider;
@@ -120,10 +125,10 @@ import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
 import org.apache.ignite.spi.discovery.DiscoverySpiDataExchange;
 import org.apache.ignite.spi.discovery.DiscoverySpiHistorySupport;
 import org.apache.ignite.spi.discovery.DiscoverySpiListener;
+import org.apache.ignite.spi.discovery.DiscoverySpiMutableCustomMessageSupport;
 import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator;
 import org.apache.ignite.spi.discovery.DiscoverySpiOrderSupport;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
 import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
@@ -478,7 +483,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
     /** {@inheritDoc} */
     @Override protected void onKernalStart0() throws IgniteCheckedException {
-        if (Boolean.TRUE.equals(ctx.config().isClientMode()) && !getSpi().isClientMode())
+        if ((getSpi() instanceof TcpDiscoverySpi) && Boolean.TRUE.equals(ctx.config().isClientMode()) && !getSpi().isClientMode())
             ctx.performance().add("Enable client mode for TcpDiscoverySpi " +
                 "(set TcpDiscoverySpi.forceServerMode to false)");
     }
@@ -551,6 +556,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             });
         }
 
+        if (ctx.config().getCommunicationFailureResolver() != null)
+            ctx.resource().injectGeneric(ctx.config().getCommunicationFailureResolver());
+
         spi.setListener(new DiscoverySpiListener() {
             private long gridStartTime;
 
@@ -559,8 +567,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 for (IgniteInClosure<ClusterNode> lsnr : locNodeInitLsnrs)
                     lsnr.apply(locNode);
 
-                if (locNode instanceof TcpDiscoveryNode) {
-                    final TcpDiscoveryNode node = (TcpDiscoveryNode)locNode;
+                if (locNode instanceof IgniteClusterNode) {
+                    final IgniteClusterNode node = (IgniteClusterNode)locNode;
 
                     if (consistentId != null)
                         node.setConsistentId(consistentId);
@@ -1052,7 +1060,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /**
      * @return Metrics provider.
      */
-    private DiscoveryMetricsProvider createMetricsProvider() {
+    public DiscoveryMetricsProvider createMetricsProvider() {
         return new DiscoveryMetricsProvider() {
             /** */
             private final long startTime = U.currentTimeMillis();
@@ -1679,13 +1687,15 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             return getSpi().pingNode(nodeId);
         }
         catch (IgniteException e) {
-            if (e.hasCause(IgniteClientDisconnectedCheckedException.class)) {
+            if (e.hasCause(IgniteClientDisconnectedCheckedException.class, IgniteClientDisconnectedException.class)) {
                 IgniteFuture<?> reconnectFut = ctx.cluster().clientReconnectFuture();
 
                 throw new IgniteClientDisconnectedCheckedException(reconnectFut, e.getMessage());
             }
 
-            throw e;
+            LT.warn(log, "Ping failed with error [node=" + nodeId + ", err=" + e + ']');
+
+            return true;
         }
         finally {
             busyLock.leaveBusy();
@@ -2025,7 +2035,16 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
         Map<Long, Collection<ClusterNode>> snapshots = topHist;
 
-        return snapshots.get(topVer);
+        Collection<ClusterNode> nodes = snapshots.get(topVer);
+
+        if (nodes == null) {
+            DiscoCache cache = discoCacheHist.get(new AffinityTopologyVersion(topVer, 0));
+
+            if (cache != null)
+                nodes = cache.allNodes();
+        }
+
+        return nodes;
     }
 
     /**
@@ -2158,6 +2177,19 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     }
 
     /**
+     * @param discoCache
+     * @param node
+     */
+    public void metricsUpdateEvent(DiscoCache discoCache, ClusterNode node) {
+        discoWrk.addEvent(EVT_NODE_METRICS_UPDATED,
+            discoCache.version(),
+            node,
+            discoCache,
+            discoCache.nodeMap.values(),
+            null);
+    }
+
+    /**
      * Gets first grid node start time, see {@link DiscoverySpi#getGridStartTime()}.
      *
      * @return Start time of the first grid node.
@@ -2211,8 +2243,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     public boolean reconnectSupported() {
         DiscoverySpi spi = getSpi();
 
-        return ctx.discovery().localNode().isClient() && (spi instanceof TcpDiscoverySpi) &&
-            !(((TcpDiscoverySpi) spi).isClientReconnectDisabled());
+        return ctx.discovery().localNode().isClient() &&
+            (spi instanceof IgniteDiscoverySpi) &&
+            ((IgniteDiscoverySpi)spi).clientReconnectSupported();
     }
 
     /**
@@ -2225,7 +2258,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
         DiscoverySpi discoverySpi = getSpi();
 
-        ((TcpDiscoverySpi)discoverySpi).reconnect();
+        ((IgniteDiscoverySpi)discoverySpi).clientReconnect();
     }
 
     /**
@@ -2379,6 +2412,76 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         cacheNodes.add(rich);
     }
 
+    /**
+     * @param cfg Configuration.
+     * @throws IgniteCheckedException If configuration is not valid.
+     */
+    public static void initCommunicationErrorResolveConfiguration(IgniteConfiguration cfg) throws IgniteCheckedException {
+        CommunicationFailureResolver rslvr = cfg.getCommunicationFailureResolver();
+        CommunicationSpi commSpi = cfg.getCommunicationSpi();
+        DiscoverySpi discoverySpi = cfg.getDiscoverySpi();
+
+        if (rslvr != null) {
+            if (!supportsCommunicationErrorResolve(commSpi))
+                throw new IgniteCheckedException("CommunicationFailureResolver is configured, but CommunicationSpi does not support communication" +
+                    "problem resolve: " + commSpi.getClass().getName());
+
+            if (!supportsCommunicationErrorResolve(discoverySpi))
+                throw new IgniteCheckedException("CommunicationFailureResolver is configured, but DiscoverySpi does not support communication" +
+                    "problem resolve: " + discoverySpi.getClass().getName());
+        }
+        else {
+            if (supportsCommunicationErrorResolve(commSpi) && supportsCommunicationErrorResolve(discoverySpi))
+                cfg.setCommunicationFailureResolver(new DefaultCommunicationFailureResolver());
+        }
+    }
+
+    /**
+     * @param spi Discovery SPI.
+     * @return {@code True} if SPI supports communication error resolve.
+     */
+    private static boolean supportsCommunicationErrorResolve(DiscoverySpi spi) {
+        return spi instanceof IgniteDiscoverySpi && ((IgniteDiscoverySpi)spi).supportsCommunicationFailureResolve();
+    }
+
+    /**
+     * @param spi Discovery SPI.
+     * @return {@code True} if SPI supports communication error resolve.
+     */
+    private static boolean supportsCommunicationErrorResolve(CommunicationSpi spi) {
+        return spi instanceof TcpCommunicationSpi;
+    }
+
+    /**
+     * @return {@code True} if communication error resolve is supported.
+     */
+    public boolean communicationErrorResolveSupported() {
+        return ctx.config().getCommunicationFailureResolver() != null;
+    }
+
+    /**
+     * @return {@code True} if configured {@link DiscoverySpi} supports mutable custom messages.
+     */
+    public boolean mutableCustomMessages() {
+        DiscoverySpiMutableCustomMessageSupport ann = U.getAnnotation(ctx.config().getDiscoverySpi().getClass(),
+            DiscoverySpiMutableCustomMessageSupport.class);
+
+        return ann != null && ann.value();
+    }
+
+    /**
+     * @param node Problem node.
+     * @param err Error.
+     */
+    public void resolveCommunicationError(ClusterNode node, Exception err) {
+        DiscoverySpi spi = getSpi();
+
+        if (!supportsCommunicationErrorResolve(spi) || !supportsCommunicationErrorResolve(ctx.config().getCommunicationSpi()))
+            throw new UnsupportedOperationException();
+
+        ((IgniteDiscoverySpi)spi).resolveCommunicationFailure(node, err);
+    }
+
     /** Worker for network segment checks. */
     private class SegmentCheckWorker extends GridWorker {
         /** */
@@ -2587,6 +2690,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
             AffinityTopologyVersion topVer = evt.get2();
 
+            if (type == EVT_NODE_METRICS_UPDATED && topVer.compareTo(discoCache.version()) < 0)
+                return;
+
             ClusterNode node = evt.get3();
 
             boolean isDaemon = node.isDaemon();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteClusterNode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteClusterNode.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteClusterNode.java
new file mode 100644
index 0000000..cbc706a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteClusterNode.java
@@ -0,0 +1,69 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Map;
+import org.apache.ignite.cache.CacheMetrics;
+import org.apache.ignite.cluster.ClusterMetrics;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ *
+ */
+public interface IgniteClusterNode extends ClusterNode {
+    /**
+     * Sets consistent globally unique node ID which survives node restarts.
+     *
+     * @param consistentId Consistent globally unique node ID.
+     */
+    public void setConsistentId(Serializable consistentId);
+
+    /**
+     * Sets node metrics.
+     *
+     * @param metrics Node metrics.
+     */
+    public void setMetrics(ClusterMetrics metrics);
+
+    /**
+     * Gets collections of cache metrics for this node. Note that node cache metrics are constantly updated
+     * and provide up to date information about caches.
+     * <p>
+     * Cache metrics are updated with some delay which is directly related to metrics update
+     * frequency. For example, by default the update will happen every {@code 2} seconds.
+     *
+     * @return Runtime metrics snapshots for this node.
+     */
+    public Map<Integer, CacheMetrics> cacheMetrics();
+
+    /**
+     * Sets node cache metrics.
+     *
+     * @param cacheMetrics Cache metrics.
+     */
+    public void setCacheMetrics(Map<Integer, CacheMetrics> cacheMetrics);
+
+    /**
+     * Whether this node is cache client (see {@link IgniteConfiguration#isClientMode()}).
+     *
+     * @return {@code True if client}.
+     */
+    public boolean isCacheClient();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpi.java
new file mode 100644
index 0000000..9aa5d14
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpi.java
@@ -0,0 +1,67 @@
+/*
+ * 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 java.util.UUID;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
+
+/**
+ *
+ */
+public interface IgniteDiscoverySpi extends DiscoverySpi {
+    /**
+     * @param nodeId Node ID.
+     * @return {@code True} if node joining or already joined topology.
+     */
+    public boolean knownNode(UUID nodeId);
+
+    /**
+     *
+     * @return {@code True} if SPI supports client reconnect.
+     */
+    public boolean clientReconnectSupported();
+
+    /**
+     *
+     */
+    public void clientReconnect();
+
+    /**
+     * For TESTING only.
+     */
+    public void simulateNodeFailure();
+
+    /**
+     * For TESTING only.
+     *
+     * @param lsnr Listener.
+     */
+    public void setInternalListener(IgniteDiscoverySpiInternalListener lsnr);
+
+    /**
+     * @return {@code True} if supports communication error resolve.
+     */
+    public boolean supportsCommunicationFailureResolve();
+
+    /**
+     * @param node Problem node.
+     * @param err Connection error.
+     */
+    public void resolveCommunicationFailure(ClusterNode node, Exception err);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpiInternalListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpiInternalListener.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpiInternalListener.java
new file mode 100644
index 0000000..24405f8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpiInternalListener.java
@@ -0,0 +1,42 @@
+/*
+ * 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.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+
+/**
+ * For TESTING only.
+ */
+public interface IgniteDiscoverySpiInternalListener {
+    /**
+     * @param locNode Local node.
+     * @param log Log.
+     */
+    public void beforeJoin(ClusterNode locNode, IgniteLogger log);
+
+    /**
+     * @param spi SPI instance.
+     * @param log Logger.
+     * @param msg Custom message.
+     * @return {@code False} to cancel event send.
+     */
+    public boolean beforeSendCustomEvent(DiscoverySpi spi, IgniteLogger log, DiscoverySpiCustomMessage msg);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserAcceptedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserAcceptedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserAcceptedMessage.java
index ef87a44..2e2aed9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserAcceptedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserAcceptedMessage.java
@@ -72,6 +72,11 @@ public class UserAcceptedMessage implements DiscoveryCustomMessage {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
         AffinityTopologyVersion topVer, DiscoCache discoCache) {
         throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserProposedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserProposedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserProposedMessage.java
index 1a0be8e..19f9e82 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserProposedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserProposedMessage.java
@@ -72,6 +72,11 @@ public class UserProposedMessage implements DiscoveryServerOnlyCustomMessage {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
         AffinityTopologyVersion topVer, DiscoCache discoCache) {
         throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinityChangeMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinityChangeMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinityChangeMessage.java
index fe1014c..937a889 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinityChangeMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinityChangeMessage.java
@@ -156,6 +156,11 @@ public class CacheAffinityChangeMessage implements DiscoveryCustomMessage {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
         AffinityTopologyVersion topVer, DiscoCache discoCache) {
         return discoCache.copy(topVer, null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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 6691b13..92b8d3e 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
@@ -1310,20 +1310,17 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
      * @param fut Current exchange future.
      * @param msg Message finish message.
      * @param resTopVer Result topology version.
-     * @throws IgniteCheckedException If failed.
      */
     public void onLocalJoin(final GridDhtPartitionsExchangeFuture fut,
         GridDhtPartitionsFullMessage msg,
-        final AffinityTopologyVersion resTopVer)
-        throws IgniteCheckedException {
+        final AffinityTopologyVersion resTopVer) {
         final Set<Integer> affReq = fut.context().groupsAffinityRequestOnJoin();
 
         final Map<Long, ClusterNode> nodesByOrder = new HashMap<>();
 
         final Map<Integer, CacheGroupAffinityMessage> joinedNodeAff = msg.joinedNodeAffinity();
 
-        assert !F.isEmpty(joinedNodeAff) : msg;
-        assert joinedNodeAff.size() >= affReq.size();
+        assert F.isEmpty(affReq) || (!F.isEmpty(joinedNodeAff) && joinedNodeAff.size() >= affReq.size()) : msg;
 
         forAllCacheGroups(false, new IgniteInClosureX<GridAffinityAssignmentCache>() {
             @Override public void applyx(GridAffinityAssignmentCache aff) throws IgniteCheckedException {
@@ -1333,7 +1330,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
 
                 assert grp != null;
 
-                if (affReq.contains(aff.groupId())) {
+                if (affReq != null && affReq.contains(aff.groupId())) {
                     assert AffinityTopologyVersion.NONE.equals(aff.lastVersion());
 
                     CacheGroupAffinityMessage affMsg = joinedNodeAff.get(aff.groupId());
@@ -2282,6 +2279,23 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
     }
 
     /**
+     * @return All registered cache groups.
+     */
+    public Map<Integer, DynamicCacheDescriptor> caches() {
+        return caches.registeredCaches;
+    }
+
+    /**
+     * @param grpId Cache group ID
+     * @return Cache affinity cache.
+     */
+    @Nullable public GridAffinityAssignmentCache groupAffinity(int grpId) {
+        CacheGroupHolder grpHolder = grpHolders.get(grpId);
+
+        return grpHolder != null ? grpHolder.affinity() : null;
+    }
+
+    /**
      *
      */
     public void dumpDebugInfo() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java
index 40bcfaf..e33256f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java
@@ -101,6 +101,11 @@ public class CacheStatisticsModeChangeMessage implements DiscoveryCustomMessage
     }
 
     /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer,
         DiscoCache discoCache) {
         throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDiscoveryMessage.java
index e35d80e..ae76c95 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDiscoveryMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDiscoveryMessage.java
@@ -173,6 +173,11 @@ public class ClientCacheChangeDiscoveryMessage implements DiscoveryCustomMessage
     }
 
     /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
         AffinityTopologyVersion topVer, DiscoCache discoCache) {
         throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDummyDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDummyDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDummyDiscoveryMessage.java
index 6ed3ecc..4ce0c87 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDummyDiscoveryMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDummyDiscoveryMessage.java
@@ -105,6 +105,11 @@ public class ClientCacheChangeDummyDiscoveryMessage implements DiscoveryCustomMe
     }
 
     /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
         AffinityTopologyVersion topVer, DiscoCache discoCache) {
         throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
index 83459a5..d85e29b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
@@ -77,6 +77,11 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer,
         DiscoCache discoCache) {
         return mgr.createDiscoCacheOnCacheChange(topVer, discoCache);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 55357ff..c2d0f42 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -82,6 +82,7 @@ import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException;
 import org.apache.ignite.internal.cluster.IgniteClusterEx;
+import org.apache.ignite.internal.managers.discovery.IgniteClusterNode;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.affinity.GridCacheAffinityImpl;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteExternalizableExpiryPolicy;
@@ -3234,7 +3235,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         List<CacheMetrics> metrics = new ArrayList<>(grp.nodes().size());
 
         for (ClusterNode node : grp.nodes()) {
-            Map<Integer, CacheMetrics> nodeCacheMetrics = ((TcpDiscoveryNode)node).cacheMetrics();
+            Map<Integer, CacheMetrics> nodeCacheMetrics = ((IgniteClusterNode)node).cacheMetrics();
 
             if (nodeCacheMetrics != null) {
                 CacheMetrics e = nodeCacheMetrics.get(context().cacheId());

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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 a30a24a..77ffce3 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
@@ -427,7 +427,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         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() + ']';
+                    "order [newOrder=" + n.order() + ", locOrder=" + loc.order() + ", evt=" + evt + ']';
 
             exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt);
 
@@ -570,12 +570,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         for (ClusterNode n : cctx.discovery().remoteNodes())
             cctx.versions().onReceived(n.id(), n.metrics().getLastDataVersion());
 
-        ClusterNode loc = cctx.localNode();
-
-        long startTime = loc.metrics().getStartTime();
-
-        assert startTime > 0;
-
         DiscoveryLocalJoinData locJoin = cctx.discovery().localJoin();
 
         GridDhtPartitionsExchangeFuture fut = null;
@@ -758,6 +752,14 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
     /**
      * @param grpId Cache group ID.
+     * @return Topology.
+     */
+    @Nullable public GridDhtPartitionTopology clientTopologyIfExists(int grpId) {
+        return clientTops.get(grpId);
+    }
+
+    /**
+     * @param grpId Cache group ID.
      * @param discoCache Discovery data cache.
      * @return Topology.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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 a3f7c94..7edac73 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
@@ -3363,7 +3363,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @return Validation result or {@code null} in case of success.
      */
     @Nullable private IgniteNodeValidationResult validateHashIdResolvers(ClusterNode node) {
-        if (!node.isClient()) {
+        if (!CU.clientNode(node)) {
             for (DynamicCacheDescriptor desc : cacheDescriptors().values()) {
                 CacheConfiguration cfg = desc.cacheConfiguration();
 
@@ -3372,7 +3372,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                     Object nodeHashObj = aff.resolveNodeHash(node);
 
-                    for (ClusterNode topNode : ctx.discovery().allNodes()) {
+                    for (ClusterNode topNode : ctx.discovery().aliveServerNodes()) {
                         Object topNodeHashObj = aff.resolveNodeHash(topNode);
 
                         if (nodeHashObj.hashCode() == topNodeHashObj.hashCode()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 83ce2ba..a5169d2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -65,6 +65,7 @@ import org.apache.ignite.internal.IgniteNodeAttributes;
 import org.apache.ignite.internal.cluster.ClusterGroupEmptyCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException;
+import org.apache.ignite.internal.managers.discovery.IgniteClusterNode;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockCancelledException;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
@@ -97,7 +98,6 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteReducer;
 import org.apache.ignite.lifecycle.LifecycleAware;
 import org.apache.ignite.plugin.CachePluginConfiguration;
-import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
@@ -1348,8 +1348,8 @@ public class GridCacheUtils {
      * @return {@code True} if given node is client node (has flag {@link IgniteConfiguration#isClientMode()} set).
      */
     public static boolean clientNode(ClusterNode node) {
-        if (node instanceof TcpDiscoveryNode)
-            return ((TcpDiscoveryNode)node).isCacheClient();
+        if (node instanceof IgniteClusterNode)
+            return ((IgniteClusterNode)node).isCacheClient();
         else
             return clientNodeDirect(node);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateFinishMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateFinishMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateFinishMessage.java
index 57f25d0..4afa403 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateFinishMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateFinishMessage.java
@@ -67,6 +67,11 @@ public class WalStateFinishMessage extends WalStateAbstractMessage {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(WalStateFinishMessage.class, this, "super", super.toString());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateProposeMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateProposeMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateProposeMessage.java
index 747fd6a..b9d96fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateProposeMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateProposeMessage.java
@@ -98,6 +98,11 @@ public class WalStateProposeMessage extends WalStateAbstractMessage {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(WalStateProposeMessage.class, this, "super", super.toString());
     }


[09/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationConnectionCheckFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationConnectionCheckFuture.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationConnectionCheckFuture.java
new file mode 100644
index 0000000..c42fa57
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationConnectionCheckFuture.java
@@ -0,0 +1,519 @@
+/*
+ * 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.spi.communication.tcp.internal;
+
+import java.net.InetSocketAddress;
+import java.nio.channels.SocketChannel;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.events.Event;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
+import org.apache.ignite.internal.util.GridLeanMap;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.nio.GridNioServer;
+import org.apache.ignite.internal.util.nio.GridNioSession;
+import org.apache.ignite.internal.util.nio.GridNioSessionMetaKey;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.spi.IgniteSpiTimeoutObject;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+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;
+
+/**
+ * Tcp Communication Connection Check Future.
+ */
+public class TcpCommunicationConnectionCheckFuture extends GridFutureAdapter<BitSet> implements IgniteSpiTimeoutObject, GridLocalEventListener {
+    /** Session future. */
+    public static final int SES_FUT_META = GridNioSessionMetaKey.nextUniqueKey();
+
+    /** */
+    private static final AtomicIntegerFieldUpdater<SingleAddressConnectFuture> connFutDoneUpdater =
+        AtomicIntegerFieldUpdater.newUpdater(SingleAddressConnectFuture.class, "done");
+
+    /** */
+    private static final AtomicIntegerFieldUpdater<MultipleAddressesConnectFuture> connResCntUpdater =
+        AtomicIntegerFieldUpdater.newUpdater(MultipleAddressesConnectFuture.class, "resCnt");
+
+    /** */
+    private final AtomicInteger resCntr = new AtomicInteger();
+
+    /** */
+    private final List<ClusterNode> nodes;
+
+    /** */
+    private volatile ConnectFuture[] futs;
+
+    /** */
+    private final GridNioServer nioSrvr;
+
+    /** */
+    private final TcpCommunicationSpi spi;
+
+    /** */
+    private final IgniteUuid timeoutObjId = IgniteUuid.randomUuid();
+
+    /** */
+    private final BitSet resBitSet;
+
+    /** */
+    private long endTime;
+
+    /** */
+    private final IgniteLogger log;
+
+    /**
+     * @param spi SPI instance.
+     * @param log Logger.
+     * @param nioSrvr NIO server.
+     * @param nodes Nodes to check.
+     */
+    public TcpCommunicationConnectionCheckFuture(TcpCommunicationSpi spi,
+        IgniteLogger log,
+        GridNioServer nioSrvr,
+        List<ClusterNode> nodes)
+    {
+        this.spi = spi;
+        this.log = log;
+        this.nioSrvr = nioSrvr;
+        this.nodes = nodes;
+
+        resBitSet = new BitSet(nodes.size());
+    }
+
+    /**
+     * @param timeout Connect timeout.
+     */
+    public void init(long timeout) {
+        ConnectFuture[] futs = new ConnectFuture[nodes.size()];
+
+        UUID locId = spi.getSpiContext().localNode().id();
+
+        for (int i = 0; i < nodes.size(); i++) {
+            ClusterNode node = nodes.get(i);
+
+            if (!node.id().equals(locId)) {
+                if (spi.getSpiContext().node(node.id()) == null) {
+                    receivedConnectionStatus(i, false);
+
+                    continue;
+                }
+
+                Collection<InetSocketAddress> addrs;
+
+                try {
+                    addrs = spi.nodeAddresses(node, false);
+                }
+                catch (Exception e) {
+                    U.error(log, "Failed to get node addresses: " + node, e);
+
+                    receivedConnectionStatus(i, false);
+
+                    continue;
+                }
+
+                if (addrs.size() == 1) {
+                    SingleAddressConnectFuture fut = new SingleAddressConnectFuture(i);
+
+                    fut.init(addrs.iterator().next(), node.id());
+
+                    futs[i] = fut;
+                }
+                else {
+                    MultipleAddressesConnectFuture fut = new MultipleAddressesConnectFuture(i);
+
+                    fut.init(addrs, node.id());
+
+                    futs[i] = fut;
+                }
+            }
+            else
+                receivedConnectionStatus(i, true);
+        }
+
+        this.futs = futs;
+
+        spi.getSpiContext().addLocalEventListener(this, EVT_NODE_LEFT, EVT_NODE_FAILED);
+
+        if (!isDone()) {
+            endTime = System.currentTimeMillis() + timeout;
+
+            spi.getSpiContext().addTimeoutObject(this);
+        }
+    }
+
+    /**
+     * @param idx Node index.
+     * @param res Success flag.
+     */
+    private void receivedConnectionStatus(int idx, boolean res) {
+        assert resCntr.get() < nodes.size();
+
+        synchronized (resBitSet) {
+            resBitSet.set(idx, res);
+        }
+
+        if (resCntr.incrementAndGet() == nodes.size())
+            onDone(resBitSet);
+    }
+
+    /**
+     * @param nodeIdx Node index.
+     * @return Node ID.
+     */
+    private UUID nodeId(int nodeIdx) {
+        return nodes.get(nodeIdx).id();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteUuid id() {
+        return timeoutObjId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long endTime() {
+        return endTime;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onEvent(Event evt) {
+        if (isDone())
+            return;
+
+        assert evt instanceof DiscoveryEvent : evt;
+        assert evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED ;
+
+        UUID nodeId = ((DiscoveryEvent)evt).eventNode().id();
+
+        for (int i = 0; i < nodes.size(); i++) {
+            if (nodes.get(i).id().equals(nodeId)) {
+                ConnectFuture fut = futs[i];
+
+                if (fut != null)
+                    fut.onNodeFailed();
+
+                return;
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onTimeout() {
+        if (isDone())
+            return;
+
+        ConnectFuture[] futs = this.futs;
+
+        for (int i = 0; i < futs.length; i++) {
+            ConnectFuture fut = futs[i];
+
+            if (fut != null)
+                fut.onTimeout();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onDone(@Nullable BitSet res, @Nullable Throwable err) {
+        if (super.onDone(res, err)) {
+            spi.getSpiContext().removeTimeoutObject(this);
+
+            spi.getSpiContext().removeLocalEventListener(this);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     *
+     */
+    private interface ConnectFuture {
+        /**
+         *
+         */
+        void onTimeout();
+
+        /**
+         *
+         */
+        void onNodeFailed();
+    }
+
+    /**
+     *
+     */
+    private class SingleAddressConnectFuture implements TcpCommunicationNodeConnectionCheckFuture, ConnectFuture {
+        /** */
+        final int nodeIdx;
+
+        /** */
+        volatile int done;
+
+        /** */
+        Map<Integer, Object> sesMeta;
+
+        /** */
+        private SocketChannel ch;
+
+        /**
+         * @param nodeIdx Node index.
+         */
+        SingleAddressConnectFuture(int nodeIdx) {
+            this.nodeIdx = nodeIdx;
+        }
+
+        /**
+         * @param addr Node address.
+         * @param rmtNodeId Id of node to open connection check session with.
+         */
+        public void init(InetSocketAddress addr, UUID rmtNodeId) {
+            boolean connect;
+
+            try {
+                ch = SocketChannel.open();
+
+                ch.configureBlocking(false);
+
+                ch.socket().setTcpNoDelay(true);
+                ch.socket().setKeepAlive(false);
+
+                connect = ch.connect(addr);
+            }
+            catch (Exception e) {
+                finish(false);
+
+                return;
+            }
+
+            if (!connect) {
+                sesMeta = new GridLeanMap<>(3);
+
+                // Set dummy key to identify connection-check outgoing connection.
+                sesMeta.put(TcpCommunicationSpi.CONN_IDX_META, new ConnectionKey(rmtNodeId, -1, -1, true));
+                sesMeta.put(SES_FUT_META, this);
+
+                nioSrvr.createSession(ch, sesMeta, true, new IgniteInClosure<IgniteInternalFuture<GridNioSession>>() {
+                    @Override public void apply(IgniteInternalFuture<GridNioSession> fut) {
+                        if (fut.error() != null)
+                            finish(false);
+                    }
+                });
+            }
+        }
+
+        /**
+         *
+         */
+        @SuppressWarnings("unchecked")
+        void cancel() {
+            if (finish(false))
+                nioSrvr.cancelConnect(ch, sesMeta);
+        }
+
+        /** {@inheritDoc} */
+        public void onTimeout() {
+            cancel();
+        }
+
+        /** {@inheritDoc} */
+        public void onConnected(UUID rmtNodeId) {
+            finish(nodeId(nodeIdx).equals(rmtNodeId));
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onNodeFailed() {
+            cancel();
+        }
+
+        /**
+         * @param res Result.
+         * @return {@code True} if result was set by this call.
+         */
+        public boolean finish(boolean res) {
+            if (connFutDoneUpdater.compareAndSet(this, 0, 1)) {
+                onStatusReceived(res);
+
+                return true;
+            }
+
+            return false;
+        }
+
+        /**
+         * @param res Result.
+         */
+        void onStatusReceived(boolean res) {
+            receivedConnectionStatus(nodeIdx, res);
+        }
+    }
+
+    /**
+     *
+     */
+    private class MultipleAddressesConnectFuture implements ConnectFuture {
+        /** */
+        volatile int resCnt;
+
+        /** */
+        volatile SingleAddressConnectFuture[] futs;
+
+        /** */
+        final int nodeIdx;
+
+        /**
+         * @param nodeIdx Node index.
+         */
+        MultipleAddressesConnectFuture(int nodeIdx) {
+            this.nodeIdx = nodeIdx;
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onNodeFailed() {
+            SingleAddressConnectFuture[] futs = this.futs;
+
+            for (int i = 0; i < futs.length; i++) {
+                ConnectFuture fut = futs[i];
+
+                if (fut != null)
+                    fut.onNodeFailed();
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onTimeout() {
+            SingleAddressConnectFuture[] futs = this.futs;
+
+            for (int i = 0; i < futs.length; i++) {
+                ConnectFuture fut = futs[i];
+
+                if (fut != null)
+                    fut.onTimeout();
+            }
+        }
+
+        /**
+         * @param addrs Node addresses.
+         * @param rmtNodeId Id of node to open connection check session with.
+         */
+        void init(Collection<InetSocketAddress> addrs, UUID rmtNodeId) {
+            SingleAddressConnectFuture[] futs = new SingleAddressConnectFuture[addrs.size()];
+
+            for (int i = 0; i < addrs.size(); i++) {
+                SingleAddressConnectFuture fut = new SingleAddressConnectFuture(nodeIdx) {
+                    @Override void onStatusReceived(boolean res) {
+                        receivedAddressStatus(res);
+                    }
+                };
+
+                futs[i] = fut;
+            }
+
+            this.futs = futs;
+
+            int idx = 0;
+
+            for (InetSocketAddress addr : addrs) {
+                futs[idx++].init(addr, rmtNodeId);
+
+                if (resCnt == Integer.MAX_VALUE)
+                    return;
+            }
+
+            // Close race.
+            if (done())
+                cancelFutures();
+        }
+
+        /**
+         * @return {@code True}
+         */
+        private boolean done() {
+            int resCnt0 = resCnt;
+
+            return resCnt0 == Integer.MAX_VALUE || resCnt0 == futs.length;
+        }
+
+        /**
+         *
+         */
+        private void cancelFutures() {
+            SingleAddressConnectFuture[] futs = this.futs;
+
+            if (futs != null) {
+                for (int i = 0; i < futs.length; i++) {
+                    SingleAddressConnectFuture fut = futs[i];
+
+                    fut.cancel();
+                }
+            }
+        }
+
+        /**
+         * @param res Result.
+         */
+        void receivedAddressStatus(boolean res) {
+            if (res) {
+                for (;;) {
+                    int resCnt0 = resCnt;
+
+                    if (resCnt0 == Integer.MAX_VALUE)
+                        return;
+
+                    if (connResCntUpdater.compareAndSet(this, resCnt0, Integer.MAX_VALUE)) {
+                        receivedConnectionStatus(nodeIdx, true);
+
+                        cancelFutures(); // Cancel others connects if they are still in progress.
+
+                        return;
+                    }
+                }
+            }
+            else {
+                for (;;) {
+                    int resCnt0 = resCnt;
+
+                    if (resCnt0 == Integer.MAX_VALUE)
+                        return;
+
+                    int resCnt1 = resCnt0 + 1;
+
+                    if (connResCntUpdater.compareAndSet(this, resCnt0, resCnt1)) {
+                        if (resCnt1 == futs.length)
+                            receivedConnectionStatus(nodeIdx, false);
+
+                        return;
+                    }
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationNodeConnectionCheckFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationNodeConnectionCheckFuture.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationNodeConnectionCheckFuture.java
new file mode 100644
index 0000000..cbf27b5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationNodeConnectionCheckFuture.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.spi.communication.tcp.internal;
+
+import java.util.UUID;
+
+/**
+ * Tcp Communication Node Connection Check Future.
+ */
+public interface TcpCommunicationNodeConnectionCheckFuture {
+    /**
+     * @param nodeId Remote node ID.
+     */
+    public void onConnected(UUID nodeId);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiCustomMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiCustomMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiCustomMessage.java
index a0f9b75..f26ad33 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiCustomMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiCustomMessage.java
@@ -30,12 +30,23 @@ import org.jetbrains.annotations.Nullable;
  */
 public interface DiscoverySpiCustomMessage extends Serializable {
     /**
-     * Called when message passed the ring.
+     * Called when custom message has been handled by all nodes.
+     *
+     * @return Ack message or {@code null} if ack is not required.
      */
     @Nullable public DiscoverySpiCustomMessage ackMessage();
 
     /**
-     * @return {@code true} if message can be modified during listener notification. Changes will be send to next nodes.
+     * @return {@code True} if message can be modified during listener notification. Changes will be send to next nodes.
      */
     public boolean isMutable();
+
+    /**
+     * Called on discovery coordinator node after listener is notified. If returns {@code true}
+     * then message is not passed to others nodes, if after this method {@link #ackMessage()} returns non-null ack
+     * message, it is sent to all nodes.
+     *
+     * @return {@code True} if message should not be sent to all nodes.
+     */
+    public boolean stopProcess();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiMutableCustomMessageSupport.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiMutableCustomMessageSupport.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiMutableCustomMessageSupport.java
new file mode 100644
index 0000000..37aa323
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiMutableCustomMessageSupport.java
@@ -0,0 +1,40 @@
+/*
+ * 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.spi.discovery;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Inherited;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * This annotation is for all implementations of {@link DiscoverySpi} that support
+ * topology mutable {@link DiscoverySpiCustomMessage}s.
+ */
+@Documented
+@Inherited
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.TYPE})
+public @interface DiscoverySpiMutableCustomMessageSupport {
+    /**
+     * @return Whether or not target SPI supports mutable {@link DiscoverySpiCustomMessage}s.
+     */
+    public boolean value();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index 2d9a314..f0a5186 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -55,6 +55,8 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.AddressResolver;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpiInternalListener;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.F;
@@ -88,6 +90,7 @@ import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
 import org.apache.ignite.spi.discovery.DiscoverySpiDataExchange;
 import org.apache.ignite.spi.discovery.DiscoverySpiHistorySupport;
 import org.apache.ignite.spi.discovery.DiscoverySpiListener;
+import org.apache.ignite.spi.discovery.DiscoverySpiMutableCustomMessageSupport;
 import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator;
 import org.apache.ignite.spi.discovery.DiscoverySpiOrderSupport;
 import org.apache.ignite.spi.discovery.tcp.internal.DiscoveryDataPacket;
@@ -103,6 +106,7 @@ import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAuthFailedMessag
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCheckFailedMessage;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryDuplicateIdMessage;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryEnsureDelivery;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryJoinRequestMessage;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_CONSISTENT_ID_BY_HOST_WITHOUT_PORT;
@@ -223,7 +227,8 @@ import static org.apache.ignite.IgniteSystemProperties.getBoolean;
 @IgniteSpiMultipleInstancesSupport(true)
 @DiscoverySpiOrderSupport(true)
 @DiscoverySpiHistorySupport(true)
-public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi {
+@DiscoverySpiMutableCustomMessageSupport(true)
+public class TcpDiscoverySpi extends IgniteSpiAdapter implements IgniteDiscoverySpi {
     /** Node attribute that is mapped to node's external addresses (value is <tt>disc.tcp.ext-addrs</tt>). */
     public static final String ATTR_EXT_ADDRS = "disc.tcp.ext-addrs";
 
@@ -409,6 +414,9 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi {
     /** */
     protected IgniteSpiContext spiCtx;
 
+    /** */
+    private IgniteDiscoverySpiInternalListener internalLsnr;
+
     /**
      * Gets current SPI state.
      *
@@ -473,6 +481,13 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi {
 
     /** {@inheritDoc} */
     @Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException {
+        IgniteDiscoverySpiInternalListener internalLsnr = this.internalLsnr;
+
+        if (internalLsnr != null) {
+            if (!internalLsnr.beforeSendCustomEvent(this, log, msg))
+                return;
+        }
+
         impl.sendCustomEvent(msg);
     }
 
@@ -1559,6 +1574,9 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi {
         OutputStream out,
         TcpDiscoveryAbstractMessage msg,
         long timeout) throws IOException, IgniteCheckedException {
+        if (internalLsnr != null && msg instanceof TcpDiscoveryJoinRequestMessage)
+            internalLsnr.beforeJoin(locNode, log);
+
         assert sock != null;
         assert msg != null;
         assert out != null;
@@ -2118,15 +2136,31 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi {
         return ignite().configuration().getSslContextFactory() != null;
     }
 
-    /**
-     * Force reconnect to cluster.
-     *
-     * @throws IgniteSpiException If failed.
-     */
-    public void reconnect() throws IgniteSpiException {
+    /** {@inheritDoc} */
+    public void clientReconnect() throws IgniteSpiException {
         impl.reconnect();
     }
 
+    /** {@inheritDoc} */
+    @Override public boolean knownNode(UUID nodeId) {
+        return getNode0(nodeId) != null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean clientReconnectSupported() {
+        return !clientReconnectDisabled;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsCommunicationFailureResolve() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void resolveCommunicationFailure(ClusterNode node, Exception err) {
+        throw new UnsupportedOperationException();
+    }
+
     /**
      * <strong>FOR TEST ONLY!!!</strong>
      */
@@ -2148,6 +2182,11 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi {
         sndMsgLsnrs.add(lsnr);
     }
 
+    /** {@inheritDoc} */
+    @Override public void setInternalListener(IgniteDiscoverySpiInternalListener lsnr) {
+        this.internalLsnr = lsnr;
+    }
+
     /**
      * <strong>FOR TEST ONLY!!!</strong>
      */
@@ -2185,7 +2224,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi {
      * <p>
      * This method is intended for test purposes only.
      */
-    protected void simulateNodeFailure() {
+    public void simulateNodeFailure() {
         impl.simulateNodeFailure();
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
index 01534f7..55fe4e6 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
@@ -33,9 +33,9 @@ import java.util.UUID;
 import org.apache.ignite.cache.CacheMetrics;
 import org.apache.ignite.cluster.ClusterMetrics;
 import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.ClusterMetricsSnapshot;
 import org.apache.ignite.internal.IgniteNodeAttributes;
+import org.apache.ignite.internal.managers.discovery.IgniteClusterNode;
 import org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -58,7 +58,7 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_NODE_CONSISTE
  * <strong>This class is not intended for public use</strong> and has been made
  * <tt>public</tt> due to certain limitations of Java technology.
  */
-public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements ClusterNode,
+public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements IgniteClusterNode,
     Comparable<TcpDiscoveryNode>, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
@@ -291,26 +291,14 @@ public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements Cluste
         return metrics;
     }
 
-    /**
-     * Sets node metrics.
-     *
-     * @param metrics Node metrics.
-     */
+    /** {@inheritDoc} */
     public void setMetrics(ClusterMetrics metrics) {
         assert metrics != null;
 
         this.metrics = metrics;
     }
 
-    /**
-     * Gets collections of cache metrics for this node. Note that node cache metrics are constantly updated
-     * and provide up to date information about caches.
-     * <p>
-     * Cache metrics are updated with some delay which is directly related to metrics update
-     * frequency. For example, by default the update will happen every {@code 2} seconds.
-     *
-     * @return Runtime metrics snapshots for this node.
-     */
+    /** {@inheritDoc} */
     public Map<Integer, CacheMetrics> cacheMetrics() {
         if (metricsProvider != null) {
             Map<Integer, CacheMetrics> cacheMetrics0 = metricsProvider.cacheMetrics();
@@ -323,11 +311,7 @@ public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements Cluste
         return cacheMetrics;
     }
 
-    /**
-     * Sets node cache metrics.
-     *
-     * @param cacheMetrics Cache metrics.
-     */
+    /** {@inheritDoc} */
     public void setCacheMetrics(Map<Integer, CacheMetrics> cacheMetrics) {
         this.cacheMetrics = cacheMetrics != null ? cacheMetrics : Collections.<Integer, CacheMetrics>emptyMap();
     }
@@ -544,11 +528,7 @@ public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements Cluste
         return node;
     }
 
-    /**
-     * Whether this node is cache client (see {@link IgniteConfiguration#isClientMode()}).
-     *
-     * @return {@code True if client}.
-     */
+    /** {@inheritDoc} */
     public boolean isCacheClient() {
         if (!cacheCliInit) {
             cacheCli = CU.clientNodeDirect(this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index f0f143d..6dc3d85 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -832,6 +832,7 @@ org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPar
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader$1
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader$2
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments
+org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtDemandedPartitionsMap
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtPartitionCountersMap
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtPartitionCountersMap2
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtPartitionHistorySuppliersMap
@@ -1129,6 +1130,7 @@ org.apache.ignite.internal.processors.closure.GridClosureProcessor$T8
 org.apache.ignite.internal.processors.closure.GridClosureProcessor$T9
 org.apache.ignite.internal.processors.closure.GridClosureProcessor$TaskNoReduceAdapter
 org.apache.ignite.internal.processors.closure.GridPeerDeployAwareTaskAdapter
+org.apache.ignite.internal.processors.cluster.ClusterNodeMetrics
 org.apache.ignite.internal.processors.cluster.BaselineTopology
 org.apache.ignite.internal.processors.cluster.BaselineTopologyHistory
 org.apache.ignite.internal.processors.cluster.BaselineTopologyHistoryItem

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityFunctionExcludeNeighborsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityFunctionExcludeNeighborsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityFunctionExcludeNeighborsAbstractSelfTest.java
index 900d4f5..eee47c7 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityFunctionExcludeNeighborsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityFunctionExcludeNeighborsAbstractSelfTest.java
@@ -32,7 +32,6 @@ import org.apache.ignite.internal.IgniteNodeAttributes;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -124,12 +123,9 @@ public abstract class AffinityFunctionExcludeNeighborsAbstractSelfTest extends G
 
                 Affinity<Object> aff = g.affinity(DEFAULT_CACHE_NAME);
 
-                List<TcpDiscoveryNode> top = new ArrayList<>();
+                List<ClusterNode> top = new ArrayList<>(g.cluster().nodes());
 
-                for (ClusterNode node : g.cluster().nodes())
-                    top.add((TcpDiscoveryNode) node);
-
-                Collections.sort(top);
+                Collections.sort((List)top);
 
                 assertEquals(grids, top.size());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlerTriggeredTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlerTriggeredTest.java b/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlerTriggeredTest.java
index 4e4d75a..5eca7d6 100644
--- a/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlerTriggeredTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlerTriggeredTest.java
@@ -120,6 +120,10 @@ public class FailureHandlerTriggeredTest extends GridCommonAbstractTest {
                 @Override public boolean isMutable() {
                     return false;
                 }
+
+                @Override public boolean stopProcess() {
+                    return false;
+                }
             });
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupHostsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupHostsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupHostsSelfTest.java
index 2328c84..141f4af 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupHostsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupHostsSelfTest.java
@@ -61,6 +61,9 @@ public class ClusterGroupHostsSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testForHosts() throws Exception {
+        if (!tcpDiscovery())
+            return;
+
         Ignite ignite = grid();
 
         assertEquals(1, ignite.cluster().forHost("h_1").nodes().size());

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java
index 9df561a..99006d1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java
@@ -68,6 +68,8 @@ public class ClusterGroupSelfTest extends ClusterGroupAbstractTest {
                 if (i == 0)
                     ignite = g;
             }
+
+            waitForTopology(NODES_CNT);
         }
         finally {
             Ignition.setClientMode(false);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsUpdateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsUpdateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsUpdateTest.java
new file mode 100644
index 0000000..6e6b4a4
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsUpdateTest.java
@@ -0,0 +1,173 @@
+/*
+ * 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;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import junit.framework.AssertionFailedError;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCompute;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cluster.ClusterGroup;
+import org.apache.ignite.cluster.ClusterMetrics;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
+import org.apache.ignite.lang.IgniteCallable;
+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;
+
+/**
+ *
+ */
+public class ClusterNodeMetricsUpdateTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        cfg.setMetricsUpdateFrequency(500);
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMetrics() throws Exception {
+        int NODES = 6;
+
+        Ignite srv0 = startGridsMultiThreaded(NODES / 2);
+
+        client = true;
+
+        startGridsMultiThreaded(NODES / 2, NODES / 2);
+
+        Map<UUID, Integer> expJobs = new HashMap<>();
+
+        for (int i = 0; i < NODES; i++)
+            expJobs.put(nodeId(i), 0);
+
+        checkMetrics(NODES, expJobs);
+
+        for (int i = 0; i < NODES; i++) {
+            UUID nodeId = nodeId(i);
+
+            IgniteCompute c = srv0.compute(srv0.cluster().forNodeId(nodeId(i)));
+
+            c.call(new DummyCallable(null));
+
+            expJobs.put(nodeId, 1);
+        }
+    }
+
+    /**
+     * @param expNodes Expected nodes.
+     * @param expJobs Expected jobs number per node.
+     */
+    private void checkMetrics0(int expNodes, Map<UUID, Integer> expJobs) {
+        List<Ignite> nodes = Ignition.allGrids();
+
+        assertEquals(expNodes, nodes.size());
+        assertEquals(expNodes, expJobs.size());
+
+        int totalJobs = 0;
+
+        for (Integer c : expJobs.values())
+            totalJobs += c;
+
+        for (final Ignite ignite : nodes) {
+            ClusterMetrics m = ignite.cluster().metrics();
+
+            assertEquals(expNodes, m.getTotalNodes());
+            assertEquals(totalJobs, m.getTotalExecutedJobs());
+
+            for (Map.Entry<UUID, Integer> e : expJobs.entrySet()) {
+                UUID nodeId = e.getKey();
+
+                ClusterGroup g = ignite.cluster().forNodeId(nodeId);
+
+                ClusterMetrics nodeM = g.metrics();
+
+                assertEquals(e.getValue(), (Integer)nodeM.getTotalExecutedJobs());
+            }
+        }
+    }
+
+    /**
+     * @param expNodes Expected nodes.
+     * @param expJobs Expected jobs number per node.
+     * @throws Exception If failed.
+     */
+    private void checkMetrics(final int expNodes, final Map<UUID, Integer> expJobs) throws Exception {
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                try {
+                    checkMetrics0(expNodes, expJobs);
+                }
+                catch (AssertionFailedError e) {
+                    return false;
+                }
+
+                return true;
+            }
+        }, 5000);
+
+        checkMetrics0(expNodes, expJobs);
+    }
+
+    /**
+     *
+     */
+    private static class DummyCallable implements IgniteCallable<Object> {
+        /** */
+        private byte[] data;
+
+        /**
+         * @param data Data.
+         */
+        DummyCallable(byte[] data) {
+            this.data = data;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object call() throws Exception {
+            return data;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/DiscoverySpiTestListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/DiscoverySpiTestListener.java b/modules/core/src/test/java/org/apache/ignite/internal/DiscoverySpiTestListener.java
new file mode 100644
index 0000000..46d9edc
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/DiscoverySpiTestListener.java
@@ -0,0 +1,162 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpiInternalListener;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.apache.ignite.testframework.GridTestUtils;
+
+/**
+ * Test callback for discovery SPI.
+ * <p>
+ * Allows block/delay node join and custom event sending.
+ */
+public class DiscoverySpiTestListener implements IgniteDiscoverySpiInternalListener {
+    /** */
+    private volatile CountDownLatch joinLatch;
+
+    /** */
+    private Set<Class<?>> blockCustomEvtCls;
+
+    /** */
+    private final Object mux = new Object();
+
+    /** */
+    private List<DiscoverySpiCustomMessage> blockedMsgs = new ArrayList<>();
+
+    /** */
+    private volatile DiscoverySpi spi;
+
+    /** */
+    private volatile IgniteLogger log;
+
+    /**
+     *
+     */
+    public void startBlockJoin() {
+        joinLatch = new CountDownLatch(1);
+    }
+
+    /**
+     *
+     */
+    public void stopBlockJoin() {
+        joinLatch.countDown();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeJoin(ClusterNode locNode, IgniteLogger log) {
+        try {
+            CountDownLatch writeLatch0 = joinLatch;
+
+            if (writeLatch0 != null) {
+                log.info("Block join");
+
+                U.await(writeLatch0);
+            }
+        }
+        catch (Exception e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean beforeSendCustomEvent(DiscoverySpi spi, IgniteLogger log, DiscoverySpiCustomMessage msg) {
+        this.spi = spi;
+        this.log = log;
+
+        synchronized (mux) {
+            if (blockCustomEvtCls != null) {
+                DiscoveryCustomMessage msg0 = GridTestUtils.getFieldValue(msg, "delegate");
+
+                if (blockCustomEvtCls.contains(msg0.getClass())) {
+                    log.info("Block custom message: " + msg0);
+
+                    blockedMsgs.add(msg);
+
+                    mux.notifyAll();
+
+                    return false;
+                }
+            }
+        }
+
+        return true;
+    }
+
+    /**
+     * @param blockCustomEvtCls Event class to block.
+     */
+    public void blockCustomEvent(Class<?> cls0, Class<?> ... blockCustomEvtCls) {
+        synchronized (mux) {
+            assert blockedMsgs.isEmpty() : blockedMsgs;
+
+            this.blockCustomEvtCls = new HashSet<>();
+
+            this.blockCustomEvtCls.add(cls0);
+
+            Collections.addAll(this.blockCustomEvtCls, blockCustomEvtCls);
+        }
+    }
+
+    /**
+     * @throws InterruptedException If interrupted.
+     */
+    public void waitCustomEvent() throws InterruptedException {
+        synchronized (mux) {
+            while (blockedMsgs.isEmpty())
+                mux.wait();
+        }
+    }
+
+    /**
+     *
+     */
+    public void stopBlockCustomEvents() {
+        if (spi == null)
+            return;
+
+        List<DiscoverySpiCustomMessage> msgs;
+
+        synchronized (this) {
+            msgs = new ArrayList<>(blockedMsgs);
+
+            blockCustomEvtCls = null;
+
+            blockedMsgs.clear();
+        }
+
+        for (DiscoverySpiCustomMessage msg : msgs) {
+            log.info("Resend blocked message: " + msg);
+
+            spi.sendCustomEvent(msg);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java
index e6b678b..883d677 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java
@@ -49,6 +49,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
 import static java.util.concurrent.TimeUnit.MINUTES;
+import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.lang.IgniteProductVersion.fromString;
@@ -158,10 +159,10 @@ public class GridDiscoverySelfTest extends GridCommonAbstractTest {
 
         final AtomicInteger cnt = new AtomicInteger();
 
-        /** Joined nodes counter. */
+        // Joined nodes counter.
         final CountDownLatch joinedCnt = new CountDownLatch(NODES_CNT);
 
-        /** Left nodes counter. */
+        // Left nodes counter.
         final CountDownLatch leftCnt = new CountDownLatch(NODES_CNT);
 
         IgnitePredicate<Event> lsnr = new IgnitePredicate<Event>() {
@@ -171,7 +172,7 @@ public class GridDiscoverySelfTest extends GridCommonAbstractTest {
 
                     joinedCnt.countDown();
                 }
-                else if (EVT_NODE_LEFT == evt.type()) {
+                else if (EVT_NODE_LEFT == evt.type() || EVT_NODE_FAILED == evt.type()) {
                     int i = cnt.decrementAndGet();
 
                     assert i >= 0;
@@ -185,7 +186,10 @@ public class GridDiscoverySelfTest extends GridCommonAbstractTest {
             }
         };
 
-        ignite.events().localListen(lsnr, EVT_NODE_LEFT, EVT_NODE_JOINED);
+        int[] evts = tcpDiscovery() ? new int[]{EVT_NODE_LEFT, EVT_NODE_JOINED} :
+            new int[]{EVT_NODE_LEFT, EVT_NODE_FAILED, EVT_NODE_JOINED};
+
+        ignite.events().localListen(lsnr, evts);
 
         try {
             for (int i = 0; i < NODES_CNT; i++)
@@ -242,6 +246,8 @@ public class GridDiscoverySelfTest extends GridCommonAbstractTest {
             for (int i = 0; i < NODES_CNT; i++)
                 stopGrid(i);
 
+            waitForTopology(1);
+
             final long topVer = discoMgr.topologyVersion();
 
             assert topVer == topVer0 + NODES_CNT * 2 : "Unexpected topology version: " + topVer;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java
index cd6b2c0..a8be541 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java
@@ -259,6 +259,8 @@ public class GridJobMasterLeaveAwareSelfTest extends GridCommonAbstractTest {
         // Now we stop master grid.
         stopGrid(lastGridIdx, true);
 
+        waitForTopology(GRID_CNT - 1);
+
         // Release communication SPI wait latches. As master node is stopped, job worker will receive and exception.
         for (int i = 0; i < lastGridIdx; i++)
             ((CommunicationSpi)grid(i).configuration().getCommunicationSpi()).releaseWaitLatch();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/GridJobStealingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridJobStealingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridJobStealingSelfTest.java
index f3a19aa..6824d51 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridJobStealingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridJobStealingSelfTest.java
@@ -187,6 +187,8 @@ public class GridJobStealingSelfTest extends GridCommonAbstractTest {
     public void testProjectionPredicateInternalStealing() throws Exception {
         final Ignite ignite3 = startGrid(3);
 
+        waitForTopology(3);
+
         final UUID node1 = ignite1.cluster().localNode().id();
         final UUID node3 = ignite3.cluster().localNode().id();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/GridSameVmStartupSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridSameVmStartupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridSameVmStartupSelfTest.java
index 66e9cf4..a04c38e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridSameVmStartupSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridSameVmStartupSelfTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal;
 import java.util.Collection;
 import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.events.DiscoveryEvent;
@@ -75,8 +76,10 @@ public class GridSameVmStartupSelfTest extends GridCommonAbstractTest {
 
             ignite2.events().localListen(new IgnitePredicate<Event>() {
                 @Override public boolean apply(Event evt) {
-                    assert evt.type() != EVT_NODE_FAILED :
-                        "Node1 did not exit gracefully.";
+                    boolean tcpDiscovery = tcpDiscovery();
+
+                    if (tcpDiscovery)
+                        assert evt.type() != EVT_NODE_FAILED : "Node1 did not exit gracefully.";
 
                     if (evt instanceof DiscoveryEvent) {
                         // Local node can send METRICS_UPDATED event.
@@ -86,8 +89,14 @@ public class GridSameVmStartupSelfTest extends GridCommonAbstractTest {
                                 ((DiscoveryEvent) evt).eventNode().id() + ", expected=" + grid1LocNodeId +
                                 ", type=" + evt.type() + ']';
 
-                        if (evt.type() == EVT_NODE_LEFT)
-                            latch.countDown();
+                        if (tcpDiscovery) {
+                            if (evt.type() == EVT_NODE_LEFT)
+                                latch.countDown();
+                        }
+                        else {
+                            if (evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED)
+                                latch.countDown();
+                        }
                     }
 
                     return true;
@@ -96,7 +105,7 @@ public class GridSameVmStartupSelfTest extends GridCommonAbstractTest {
 
             stopGrid(1);
 
-            latch.await();
+            assertTrue(latch.await(10, TimeUnit.SECONDS));
 
             Collection<ClusterNode> top2 = ignite2.cluster().forRemotes().nodes();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java
index 7e368cb..f71ffb0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java
@@ -45,6 +45,8 @@ public class GridSelfTest extends ClusterGroupAbstractTest {
 
         for (int i = 0; i < NODES_CNT; i++)
             startGrid(i);
+
+        waitForTopology(NODES_CNT);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java
index fa9cc35..e68ea13 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.net.Socket;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
@@ -38,6 +39,7 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.managers.communication.GridIoMessage;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteFuture;
@@ -143,6 +145,14 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
 
     /**
      * @param ignite Node.
+     * @return Discovery SPI.
+     */
+    protected static IgniteDiscoverySpi spi0(Ignite ignite) {
+        return ((IgniteDiscoverySpi)ignite.configuration().getDiscoverySpi());
+    }
+
+    /**
+     * @param ignite Node.
      * @return Communication SPI.
      */
     protected BlockTcpCommunicationSpi commSpi(Ignite ignite) {
@@ -185,16 +195,28 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
      * @return Server node client connected to.
      */
     protected Ignite clientRouter(Ignite client) {
-        TcpDiscoveryNode node = (TcpDiscoveryNode)client.cluster().localNode();
+        if (tcpDiscovery()) {
+            TcpDiscoveryNode node = (TcpDiscoveryNode)client.cluster().localNode();
+
+            assertTrue(node.isClient());
+            assertNotNull(node.clientRouterNodeId());
 
-        assertTrue(node.isClient());
-        assertNotNull(node.clientRouterNodeId());
+            Ignite srv = G.ignite(node.clientRouterNodeId());
 
-        Ignite srv = G.ignite(node.clientRouterNodeId());
+            assertNotNull(srv);
+
+            return srv;
+        }
+        else {
+            for (Ignite node : G.allGrids()) {
+                if (!node.cluster().localNode().isClient())
+                    return node;
+            }
 
-        assertNotNull(srv);
+            fail();
 
-        return srv;
+            return null;
+        }
     }
 
     /**
@@ -251,15 +273,24 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
         List<Ignite> clients, Ignite srv,
         @Nullable Runnable disconnectedC)
         throws Exception {
-        final TestTcpDiscoverySpi srvSpi = spi(srv);
+        final IgniteDiscoverySpi srvSpi = spi0(srv);
 
         final CountDownLatch disconnectLatch = new CountDownLatch(clients.size());
         final CountDownLatch reconnectLatch = new CountDownLatch(clients.size());
 
         log.info("Block reconnect.");
 
-        for (Ignite client : clients)
-            spi(client).writeLatch = new CountDownLatch(1);
+        List<DiscoverySpiTestListener> blockLsnrs = new ArrayList<>();
+
+        for (Ignite client : clients) {
+            DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
+
+            lsnr.startBlockJoin();
+
+            blockLsnrs.add(lsnr);
+
+            spi0(client).setInternalListener(lsnr);
+        }
 
         IgnitePredicate<Event> p = new IgnitePredicate<Event>() {
             @Override public boolean apply(Event evt) {
@@ -291,8 +322,8 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
 
         log.info("Allow reconnect.");
 
-        for (Ignite client : clients)
-            spi(client).writeLatch.countDown();
+        for (DiscoverySpiTestListener blockLsnr : blockLsnrs)
+            blockLsnr.stopBlockJoin();
 
         waitReconnectEvent(log, reconnectLatch);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectApiExceptionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectApiExceptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectApiExceptionTest.java
index 06bde99..43da2d1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectApiExceptionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectApiExceptionTest.java
@@ -44,6 +44,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.CollectionConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.Event;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
 import org.apache.ignite.internal.util.typedef.C1;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -51,6 +52,7 @@ import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.resources.IgniteInstanceResource;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
 import org.apache.ignite.testframework.GridTestUtils;
 
 import static java.util.concurrent.TimeUnit.SECONDS;
@@ -99,7 +101,7 @@ public class IgniteClientReconnectApiExceptionTest extends IgniteClientReconnect
      * @throws Exception If failed.
      */
     @SuppressWarnings("unchecked")
-    public void dataStructureOperationsTest() throws Exception {
+    private void dataStructureOperationsTest() throws Exception {
         clientMode = true;
 
         final Ignite client = startGrid(serverCount());
@@ -219,7 +221,7 @@ public class IgniteClientReconnectApiExceptionTest extends IgniteClientReconnect
      * @throws Exception If failed.
      */
     @SuppressWarnings("unchecked")
-    public void cacheOperationsTest() throws Exception {
+    private void cacheOperationsTest() throws Exception {
         clientMode = true;
 
         final Ignite client = startGrid(serverCount());
@@ -537,7 +539,7 @@ public class IgniteClientReconnectApiExceptionTest extends IgniteClientReconnect
      * @throws Exception If failed.
      */
     @SuppressWarnings("unchecked")
-    public void igniteOperationsTest() throws Exception {
+    private void igniteOperationsTest() throws Exception {
         clientMode = true;
 
         final Ignite client = startGrid(serverCount());
@@ -775,11 +777,11 @@ public class IgniteClientReconnectApiExceptionTest extends IgniteClientReconnect
         throws Exception {
         assertNotNull(client.cache(DEFAULT_CACHE_NAME));
 
-        final TestTcpDiscoverySpi clientSpi = spi(client);
+        final IgniteDiscoverySpi clientSpi = spi0(client);
 
         Ignite srv = clientRouter(client);
 
-        TestTcpDiscoverySpi srvSpi = spi(srv);
+        DiscoverySpi srvSpi = spi0(srv);
 
         final CountDownLatch disconnectLatch = new CountDownLatch(1);
 
@@ -787,7 +789,10 @@ public class IgniteClientReconnectApiExceptionTest extends IgniteClientReconnect
 
         log.info("Block reconnect.");
 
-        clientSpi.writeLatch = new CountDownLatch(1);
+        DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
+
+        clientSpi.setInternalListener(lsnr);
+        lsnr.startBlockJoin();
 
         final List<IgniteInternalFuture> futs = new ArrayList<>();
 
@@ -832,7 +837,7 @@ public class IgniteClientReconnectApiExceptionTest extends IgniteClientReconnect
 
             log.info("Allow reconnect.");
 
-            clientSpi.writeLatch.countDown();
+            lsnr.stopBlockJoin();
 
             waitReconnectEvent(reconnectLatch);
 
@@ -857,7 +862,7 @@ public class IgniteClientReconnectApiExceptionTest extends IgniteClientReconnect
             }
         }
         finally {
-            clientSpi.writeLatch.countDown();
+            lsnr.stopBlockJoin();
 
             for (IgniteInternalFuture fut : futs)
                 fut.cancel();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java
index 00daf5f..d1e3ade 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java
@@ -111,7 +111,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         IgniteAtomicSequence clientAtomicSeq = client.atomicSequence("atomicSeq", 0, true);
 
@@ -144,7 +144,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         final IgniteAtomicSequence clientAtomicSeq = client.atomicSequence("atomicSeqRmv", 0, true);
 
@@ -192,7 +192,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         BlockTcpCommunicationSpi commSpi = commSpi(srv);
 
@@ -253,7 +253,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         IgniteAtomicReference<String> clientAtomicRef = client.atomicReference("atomicRef", "1st value", true);
 
@@ -294,7 +294,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         final IgniteAtomicReference<String> clientAtomicRef =
             client.atomicReference("atomicRefRemoved", "1st value", true);
@@ -347,7 +347,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         final IgniteAtomicReference<String> clientAtomicRef =
             client.atomicReference("atomicRefInProg", "1st value", true);
@@ -414,7 +414,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         IgniteAtomicStamped clientAtomicStamped = client.atomicStamped("atomicStamped", 0, 0, true);
 
@@ -455,7 +455,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         final IgniteAtomicStamped clientAtomicStamped = client.atomicStamped("atomicStampedRemoved", 0, 0, true);
 
@@ -506,7 +506,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         final IgniteAtomicStamped clientAtomicStamped = client.atomicStamped("atomicStampedInProgress", 0, 0, true);
 
@@ -574,7 +574,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         IgniteAtomicLong clientAtomicLong = client.atomicLong("atomicLong", 0, true);
 
@@ -605,7 +605,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         final IgniteAtomicLong clientAtomicLong = client.atomicLong("atomicLongRmv", 0, true);
 
@@ -646,7 +646,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         BlockTcpCommunicationSpi commSpi = commSpi(srv);
 
@@ -701,7 +701,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         IgniteCountDownLatch clientLatch = client.countDownLatch("latch1", 3, false, true);
 
@@ -742,7 +742,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         IgniteSemaphore clientSemaphore = client.semaphore("semaphore1", 3, false, true);
 
@@ -789,7 +789,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         IgniteLock clientLock = client.reentrantLock("lock1", true, fair, true);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
index 518e674..3cb82e0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
@@ -49,6 +49,7 @@ import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.managers.communication.GridIoMessage;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetResponse;
@@ -67,6 +68,7 @@ import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.resources.LoggerResource;
 import org.apache.ignite.spi.IgniteSpiException;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.transactions.Transaction;
@@ -155,11 +157,11 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
 
         IgniteEx client = startGrid(SRV_CNT);
 
-        final TestTcpDiscoverySpi clientSpi = spi(client);
+        final IgniteDiscoverySpi clientSpi = spi0(client);
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
-        TestTcpDiscoverySpi srvSpi = spi(srv);
+        DiscoverySpi srvSpi = ignite(0).configuration().getDiscoverySpi();
 
         final IgniteCache<Object, Object> cache = client.getOrCreateCache(new CacheConfiguration<>(DEFAULT_CACHE_NAME));
 
@@ -188,7 +190,11 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
 
         log.info("Block reconnect.");
 
-        clientSpi.writeLatch = new CountDownLatch(1);
+        DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
+
+        clientSpi.setInternalListener(lsnr);
+
+        lsnr.startBlockJoin();
 
         final AtomicReference<IgniteInternalFuture> blockPutRef = new AtomicReference<>();
 
@@ -254,7 +260,7 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
 
         log.info("Allow reconnect.");
 
-        clientSpi.writeLatch.countDown();
+        lsnr.stopBlockJoin();
 
         assertTrue(reconnectLatch.await(5000, MILLISECONDS));
 
@@ -319,7 +325,7 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
 
         IgniteEx client = startGrid(SRV_CNT);
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME);
 
@@ -412,17 +418,21 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
         final TransactionConcurrency txConcurrency,
         final IgniteCache<Object, Object> cache)
         throws Exception {
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
-        final TestTcpDiscoverySpi clientSpi = spi(client);
-        final TestTcpDiscoverySpi srvSpi = spi(srv);
+        final IgniteDiscoverySpi clientSpi = spi0(client);
+        final DiscoverySpi srvSpi = spi0(srv);
 
         final CountDownLatch disconnectLatch = new CountDownLatch(1);
         final CountDownLatch reconnectLatch = new CountDownLatch(1);
 
         log.info("Block reconnect.");
 
-        clientSpi.writeLatch = new CountDownLatch(1);
+        DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
+
+        clientSpi.setInternalListener(lsnr);
+
+        lsnr.startBlockJoin();
 
         client.events().localListen(new IgnitePredicate<Event>() {
             @Override public boolean apply(Event evt) {
@@ -530,7 +540,7 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
 
         assertTrue(putFailed.await(5000, MILLISECONDS));
 
-        clientSpi.writeLatch.countDown();
+        lsnr.stopBlockJoin();
 
         waitReconnectEvent(reconnectLatch);
 
@@ -604,9 +614,9 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
 
         IgniteEx client = startGrid(SRV_CNT);
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
-        TestTcpDiscoverySpi srvSpi = spi(srv);
+        DiscoverySpi srvSpi = spi0(srv);
 
         TestCommunicationSpi coordCommSpi = (TestCommunicationSpi)grid(0).configuration().getCommunicationSpi();
 
@@ -691,7 +701,7 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
         IgniteInternalFuture<Boolean> fut = GridTestUtils.runAsync(new Callable<Boolean>() {
             @Override public Boolean call() throws Exception {
                 try {
-                    Ignition.start(optimize(getConfiguration(getTestIgniteInstanceName(SRV_CNT))));
+                    startGrid(optimize(getConfiguration(getTestIgniteInstanceName(SRV_CNT))));
 
                     // Commented due to IGNITE-4473, because
                     // IgniteClientDisconnectedException won't
@@ -722,7 +732,7 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
             }
         });
 
-        TestTcpDiscoverySpi srvSpi = spi(srv);
+        DiscoverySpi srvSpi = spi0(srv);
 
         try {
             if (!joinLatch.await(10_000, MILLISECONDS)) {
@@ -1256,30 +1266,35 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
      *
      */
     static class TestClass1 implements Serializable {
+        // No-op.
     }
 
     /**
      *
      */
     static class TestClass2 implements Serializable {
+        // No-op.
     }
 
     /**
      *
      */
     static class TestClass3 implements Serializable {
+        // No-op.
     }
 
     /**
      *
      */
     static class TestClass4 implements Serializable {
+        // No-op.
     }
 
     /**
      *
      */
     static class TestClass5 implements Serializable {
+        // No-op.
     }
 
     /**
@@ -1294,11 +1309,11 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
         Class<?> msgToBlock,
         final IgniteInClosure<IgniteCache<Object, Object>> c)
         throws Exception {
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         final UUID id = client.localNode().id();
 
-        TestTcpDiscoverySpi srvSpi = spi(srv);
+        DiscoverySpi srvSpi = spi0(srv);
 
         final IgniteCache<Object, Object> cache = client.getOrCreateCache(ccfg);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
index 3f0e33d..5be59b0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
@@ -180,7 +180,7 @@ public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectA
     private void serverNodeReconnect(CollectionConfiguration colCfg) throws Exception {
         final Ignite client = grid(serverCount());
 
-        final Ignite srv = clientRouter(client);
+        final Ignite srv = ignite(0);
 
         assertNotNull(srv.queue("q", 0, colCfg));
         assertNotNull(srv.set("s", colCfg));
@@ -201,7 +201,7 @@ public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectA
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         final String setName = "set-" + colCfg.getAtomicityMode();
 
@@ -235,7 +235,7 @@ public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectA
 
         assertTrue(client.cluster().localNode().isClient());
 
-        final Ignite srv = clientRouter(client);
+        final Ignite srv = ignite(0);
 
         final String setName = "set-rm-" + colCfg.getAtomicityMode();
 
@@ -281,7 +281,7 @@ public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectA
 
         assertTrue(client.cluster().localNode().isClient());
 
-        final Ignite srv = clientRouter(client);
+        final Ignite srv = ignite(0);
 
         final String setName = "set-in-progress-" + colCfg.getAtomicityMode();
 
@@ -347,7 +347,7 @@ public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectA
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         final String setName = "queue-" + colCfg.getAtomicityMode();
 
@@ -379,7 +379,7 @@ public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectA
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         final String setName = "queue-rmv" + colCfg.getAtomicityMode();
 
@@ -423,7 +423,7 @@ public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectA
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         final String setName = "queue-rmv" + colCfg.getAtomicityMode();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectComputeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectComputeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectComputeTest.java
index cce0c7e..57d3188 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectComputeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectComputeTest.java
@@ -49,7 +49,7 @@ public class IgniteClientReconnectComputeTest extends IgniteClientReconnectAbstr
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         IgniteCache<Integer, Integer> cache = client.getOrCreateCache("test-cache");
 
@@ -103,7 +103,7 @@ public class IgniteClientReconnectComputeTest extends IgniteClientReconnectAbstr
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         BlockTcpCommunicationSpi commSpi = commSpi(srv);
 
@@ -152,7 +152,7 @@ public class IgniteClientReconnectComputeTest extends IgniteClientReconnectAbstr
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         BlockTcpCommunicationSpi commSpi = commSpi(srv);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
index ca0d889..d68fc1c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
@@ -28,6 +28,7 @@ import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
 import org.apache.ignite.internal.util.typedef.P2;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteRunnable;
@@ -61,9 +62,9 @@ public class IgniteClientReconnectContinuousProcessorTest extends IgniteClientRe
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
-        TestTcpDiscoverySpi srvSpi = spi(srv);
+        IgniteDiscoverySpi srvSpi = spi0(srv);
 
         EventListener lsnr = new EventListener();
 
@@ -133,9 +134,9 @@ public class IgniteClientReconnectContinuousProcessorTest extends IgniteClientRe
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
-        TestTcpDiscoverySpi srvSpi = spi(srv);
+        IgniteDiscoverySpi srvSpi = spi0(srv);
 
         final String topic = "testTopic";
 
@@ -309,9 +310,9 @@ public class IgniteClientReconnectContinuousProcessorTest extends IgniteClientRe
         CacheEventListener lsnr)
         throws Exception
     {
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
-        TestTcpDiscoverySpi srvSpi = spi(srv);
+        IgniteDiscoverySpi srvSpi = spi0(srv);
 
         final CountDownLatch reconnectLatch = new CountDownLatch(1);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectDiscoveryStateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectDiscoveryStateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectDiscoveryStateTest.java
index c071ee2..6e77742 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectDiscoveryStateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectDiscoveryStateTest.java
@@ -27,6 +27,7 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
 
 import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED;
 import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_RECONNECTED;
@@ -64,20 +65,23 @@ public class IgniteClientReconnectDiscoveryStateTest extends IgniteClientReconne
         nodeCnt.put(1, 1);
         nodeCnt.put(2, 2);
         nodeCnt.put(3, 3);
-        nodeCnt.put(4, 4);
 
-        for (Map.Entry<Integer, Integer> e : nodeCnt.entrySet()) {
-            Collection<ClusterNode> nodes = cluster.topology(e.getKey());
+        if (tcpDiscovery()) {
+            nodeCnt.put(4, 4);
 
-            assertNotNull("No nodes for topology: " + e.getKey(), nodes);
-            assertEquals((int)e.getValue(), nodes.size());
+            for (Map.Entry<Integer, Integer> e : nodeCnt.entrySet()) {
+                Collection<ClusterNode> nodes = cluster.topology(e.getKey());
+
+                assertNotNull("No nodes for topology: " + e.getKey(), nodes);
+                assertEquals((int)e.getValue(), nodes.size());
+            }
         }
 
         ClusterNode locNode = cluster.localNode();
 
         assertEquals(topVer, locNode.order());
 
-        TestTcpDiscoverySpi srvSpi = spi(clientRouter(client));
+        DiscoverySpi srvSpi = ignite(0).configuration().getDiscoverySpi();
 
         final CountDownLatch reconnectLatch = new CountDownLatch(1);
 
@@ -112,7 +116,11 @@ public class IgniteClientReconnectDiscoveryStateTest extends IgniteClientReconne
         assertEquals(topVer, locNode.order());
         assertEquals(topVer, cluster.topologyVersion());
 
-        nodeCnt.put(5, 3);
+        if (tcpDiscovery())
+            nodeCnt.put(5, 3);
+        else
+            nodeCnt.clear();
+
         nodeCnt.put(6, 4);
 
         for (Map.Entry<Integer, Integer> e : nodeCnt.entrySet()) {


[03/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ZookeeperNodeStart.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ZookeeperNodeStart.java b/modules/zookeeper/src/test/java/org/apache/ZookeeperNodeStart.java
new file mode 100644
index 0000000..ef4d5f4
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ZookeeperNodeStart.java
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+import org.apache.ignite.Ignition;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi;
+
+/**
+ *
+ */
+public class ZookeeperNodeStart {
+    public static void main(String[] args) throws Exception {
+        try {
+            IgniteConfiguration cfg = new IgniteConfiguration();
+
+            ZookeeperDiscoverySpi spi = new ZookeeperDiscoverySpi();
+
+            spi.setZkConnectionString("localhost:2181");
+
+            cfg.setDiscoverySpi(spi);
+
+            Ignition.start(cfg);
+        }
+        catch (Throwable e) {
+            e.printStackTrace(System.out);
+
+            System.exit(1);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/IgniteCacheEntryListenerWithZkDiscoAtomicTest.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/IgniteCacheEntryListenerWithZkDiscoAtomicTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/IgniteCacheEntryListenerWithZkDiscoAtomicTest.java
new file mode 100644
index 0000000..754a6bf
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/IgniteCacheEntryListenerWithZkDiscoAtomicTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.spi.discovery.zk;
+
+import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerAtomicTest;
+
+/**
+ * Class is added to mute {@link #testConcurrentRegisterDeregister} test in ZooKeeper suite
+ * (see related ticket).
+ *
+ * When slow down is tracked down and fixed this class can be replaced back with its parent.
+ */
+public class IgniteCacheEntryListenerWithZkDiscoAtomicTest extends IgniteCacheEntryListenerAtomicTest {
+    /** {@inheritDoc} */
+    @Override public void testConcurrentRegisterDeregister() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-8109");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiAbstractTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiAbstractTestSuite.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiAbstractTestSuite.java
new file mode 100644
index 0000000..766635c
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiAbstractTestSuite.java
@@ -0,0 +1,118 @@
+/*
+ * 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.spi.discovery.zk;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+import junit.framework.TestSuite;
+import org.apache.curator.test.InstanceSpec;
+import org.apache.curator.test.TestingCluster;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.testframework.config.GridTestProperties;
+
+/**
+ * Allows to run regular Ignite tests with {@link org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi}.
+ */
+public abstract class ZookeeperDiscoverySpiAbstractTestSuite extends TestSuite {
+    /** */
+    private static TestingCluster testingCluster;
+
+    /**
+     * @throws Exception If failed.
+     */
+    public static void initSuite() throws Exception {
+        System.setProperty("zookeeper.forceSync", "false");
+
+        testingCluster = createTestingCluster(3);
+
+        testingCluster.start();
+
+        System.setProperty(GridTestProperties.IGNITE_CFG_PREPROCESSOR_CLS, ZookeeperDiscoverySpiAbstractTestSuite.class.getName());
+    }
+
+    /**
+     * Called via reflection by {@link org.apache.ignite.testframework.junits.GridAbstractTest}.
+     *
+     * @param cfg Configuration to change.
+     */
+    public synchronized static void preprocessConfiguration(IgniteConfiguration cfg) {
+        if (testingCluster == null)
+            throw new IllegalStateException("Test Zookeeper cluster is not started.");
+
+        ZookeeperDiscoverySpi zkSpi = new ZookeeperDiscoverySpi();
+
+        DiscoverySpi spi = cfg.getDiscoverySpi();
+
+        if (spi instanceof TcpDiscoverySpi)
+            zkSpi.setClientReconnectDisabled(((TcpDiscoverySpi)spi).isClientReconnectDisabled());
+
+        zkSpi.setSessionTimeout(30_000);
+        zkSpi.setZkConnectionString(testingCluster.getConnectString());
+
+        cfg.setDiscoverySpi(zkSpi);
+    }
+
+    /**
+     * @param instances Number of instances in
+     * @return Test cluster.
+     */
+    public static TestingCluster createTestingCluster(int instances) {
+        String tmpDir = System.getProperty("java.io.tmpdir");
+
+        List<InstanceSpec> specs = new ArrayList<>();
+
+        for (int i = 0; i < instances; i++) {
+            File file = new File(tmpDir, "apacheIgniteTestZk-" + i);
+
+            if (file.isDirectory())
+                deleteRecursively0(file);
+            else {
+                if (!file.mkdirs())
+                    throw new IgniteException("Failed to create directory for test Zookeeper server: " + file.getAbsolutePath());
+            }
+
+            specs.add(new InstanceSpec(file, -1, -1, -1, true, -1, -1, 500));
+        }
+
+        return new TestingCluster(specs);
+    }
+
+    /**
+     * @param file File or directory to delete.
+     */
+    private static void deleteRecursively0(File file) {
+        File[] files = file.listFiles();
+
+        if (files == null)
+            return;
+
+        for (File f : files) {
+            if (f.isDirectory())
+                deleteRecursively0(f);
+            else {
+                if (!f.delete())
+                    throw new IgniteException("Failed to delete file: " + f.getAbsolutePath());
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite1.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite1.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite1.java
new file mode 100644
index 0000000..860488b
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite1.java
@@ -0,0 +1,44 @@
+/*
+ * 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.spi.discovery.zk;
+
+import junit.framework.TestSuite;
+import org.apache.ignite.spi.discovery.zk.internal.ZookeeperClientTest;
+import org.apache.ignite.spi.discovery.zk.internal.ZookeeperDiscoverySpiSaslSuccessfulAuthTest;
+import org.apache.ignite.spi.discovery.zk.internal.ZookeeperDiscoverySpiTest;
+
+/**
+ *
+ */
+public class ZookeeperDiscoverySpiTestSuite1 extends TestSuite {
+    /**
+     * @return Test suite.
+     * @throws Exception Thrown in case of the failure.
+     */
+    public static TestSuite suite() throws Exception {
+        System.setProperty("zookeeper.forceSync", "false");
+
+        TestSuite suite = new TestSuite("ZookeeperDiscoverySpi Test Suite");
+
+        suite.addTestSuite(ZookeeperClientTest.class);
+        suite.addTestSuite(ZookeeperDiscoverySpiTest.class);
+        suite.addTestSuite(ZookeeperDiscoverySpiSaslSuccessfulAuthTest.class);
+
+        return suite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite2.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite2.java
new file mode 100644
index 0000000..3775aa1
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite2.java
@@ -0,0 +1,94 @@
+/*
+ * 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.spi.discovery.zk;
+
+import junit.framework.TestSuite;
+import org.apache.curator.test.TestingCluster;
+import org.apache.ignite.internal.ClusterNodeMetricsUpdateTest;
+import org.apache.ignite.internal.IgniteClientReconnectCacheTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerAtomicTest;
+import org.apache.ignite.internal.processors.cache.datastructures.IgniteClientDataStructuresTest;
+import org.apache.ignite.internal.processors.cache.datastructures.partitioned.GridCachePartitionedNodeRestartTxSelfTest;
+import org.apache.ignite.internal.processors.cache.datastructures.partitioned.GridCachePartitionedSequenceApiSelfTest;
+import org.apache.ignite.internal.processors.cache.datastructures.replicated.GridCacheReplicatedSequenceApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePutRetryAtomicSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePutRetryTransactionalSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicMultiNodeFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedNodeRestartTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedAtomicMultiNodeFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedMultiNodeFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedNodeRestartSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicMultiJvmFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.multijvm.GridCachePartitionedMultiJvmFullApiSelfTest;
+import org.apache.ignite.internal.processors.continuous.GridEventConsumeSelfTest;
+
+/**
+ * Regular Ignite tests executed with {@link org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi}.
+ */
+public class ZookeeperDiscoverySpiTestSuite2 extends ZookeeperDiscoverySpiAbstractTestSuite {
+    /** */
+    private static TestingCluster testingCluster;
+
+    /**
+     * @return Test suite.
+     * @throws Exception Thrown in case of the failure.
+     */
+    public static TestSuite suite() throws Exception {
+        System.setProperty("H2_JDBC_CONNECTIONS", "500"); // For multi-jvm tests.
+
+        initSuite();
+
+        TestSuite suite = new TestSuite("ZookeeperDiscoverySpi Test Suite");
+
+        suite.addTestSuite(ZookeeperDiscoverySuitePreprocessorTest.class);
+
+        suite.addTestSuite(GridCacheReplicatedNodeRestartSelfTest.class);
+        suite.addTestSuite(GridCachePartitionedNodeRestartTest.class);
+
+        suite.addTestSuite(IgniteCacheEntryListenerWithZkDiscoAtomicTest.class);
+
+        suite.addTestSuite(GridEventConsumeSelfTest.class);
+
+        suite.addTestSuite(IgniteClientReconnectCacheTest.class);
+
+        suite.addTestSuite(IgniteCachePutRetryAtomicSelfTest.class);
+        suite.addTestSuite(IgniteCachePutRetryTransactionalSelfTest.class);
+
+        suite.addTestSuite(ClusterNodeMetricsUpdateTest.class);
+
+        suite.addTestSuite(GridCachePartitionedMultiNodeFullApiSelfTest.class);
+        suite.addTestSuite(GridCacheReplicatedMultiNodeFullApiSelfTest.class);
+
+        suite.addTestSuite(GridCacheAtomicMultiNodeFullApiSelfTest.class);
+        suite.addTestSuite(GridCacheReplicatedAtomicMultiNodeFullApiSelfTest.class);
+
+        suite.addTestSuite(GridCachePartitionedNodeRestartTxSelfTest.class);
+        suite.addTestSuite(IgniteClientDataStructuresTest.class);
+        suite.addTestSuite(GridCacheReplicatedSequenceApiSelfTest.class);
+        suite.addTestSuite(GridCachePartitionedSequenceApiSelfTest.class);
+
+        suite.addTestSuite(IgniteCacheReplicatedQuerySelfTest.class);
+
+        suite.addTestSuite(GridCacheAtomicMultiJvmFullApiSelfTest.class);
+        suite.addTestSuite(GridCachePartitionedMultiJvmFullApiSelfTest.class);
+
+        return suite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySuitePreprocessorTest.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySuitePreprocessorTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySuitePreprocessorTest.java
new file mode 100644
index 0000000..28cf17f
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySuitePreprocessorTest.java
@@ -0,0 +1,101 @@
+/*
+ * 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.spi.discovery.zk;
+
+import java.util.List;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
+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.config.GridTestProperties;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Sanity test verifying that configuration callback specified via
+ * {@link GridTestProperties#IGNITE_CFG_PREPROCESSOR_CLS} really works.
+ * <p>
+ * This test should be run as part of {@link ZookeeperDiscoverySpiTestSuite2}.
+ */
+public class ZookeeperDiscoverySuitePreprocessorTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        // Test sets TcpDiscoverySpi, but it should be automatically changed to ZookeeperDiscoverySpi.
+        TcpDiscoverySpi spi = new TcpDiscoverySpi();
+
+        spi.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(spi);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        super.afterTest();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSpiConfigurationIsChanged() throws Exception {
+        startGrid(0);
+
+        checkDiscoverySpi(1);
+
+        startGrid(1);
+
+        checkDiscoverySpi(2);
+
+        startGridsMultiThreaded(2, 2);
+
+        checkDiscoverySpi(4);
+
+        startGrid();
+
+        checkDiscoverySpi(5);
+    }
+
+    /**
+     * @param expNodes Expected nodes number.
+     * @throws Exception If failed.
+     */
+    private void checkDiscoverySpi(int expNodes) throws Exception {
+        List<Ignite> nodes = G.allGrids();
+
+        assertEquals(expNodes, nodes.size());
+
+        for (Ignite node : nodes) {
+            DiscoverySpi spi = node.configuration().getDiscoverySpi();
+
+            assertTrue("Node should be started with " + ZookeeperDiscoverySpi.class.getName(),
+                    spi instanceof ZookeeperDiscoverySpi);
+        }
+
+        waitForTopology(expNodes);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientTest.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientTest.java
new file mode 100644
index 0000000..e7cb97a
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientTest.java
@@ -0,0 +1,495 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.curator.test.TestingCluster;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ *
+ */
+public class ZookeeperClientTest extends GridCommonAbstractTest {
+    /** */
+    private static final int SES_TIMEOUT = 60_000;
+
+    /** */
+    private TestingCluster zkCluster;
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        closeZK();
+
+        super.afterTest();
+    }
+
+    /**
+     * @param sesTimeout Session timeout.
+     * @return Client.
+     * @throws Exception If failed.
+     */
+    private ZookeeperClient createClient(int sesTimeout) throws Exception {
+        return new ZookeeperClient(log, zkCluster.getConnectString(), sesTimeout, null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSaveLargeValue() throws Exception {
+        startZK(1);
+
+        final ZookeeperClient client = createClient(SES_TIMEOUT);
+
+        byte[] data = new byte[1024 * 1024];
+
+        String basePath = "/ignite";
+
+        assertTrue(client.needSplitNodeData(basePath, data, 2));
+
+        List<byte[]> parts = client.splitNodeData(basePath, data, 2);
+
+        assertTrue(parts.size() > 1);
+
+        ZooKeeper zk = client.zk();
+
+        for (int i = 0; i < parts.size(); i++) {
+            byte[] part = parts.get(i);
+
+            assertTrue(part.length > 0);
+
+            String path0 = basePath + ":" + i;
+
+            zk.create(path0, part, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClose() throws Exception {
+        startZK(1);
+
+        final ZookeeperClient client = createClient(SES_TIMEOUT);
+
+        client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT);
+
+        client.zk().close();
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT);
+
+                return null;
+            }
+        }, ZookeeperClientFailedException.class, null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCreateAll() throws Exception {
+        startZK(1);
+
+        ZookeeperClient client = createClient(SES_TIMEOUT);
+
+        client.createIfNeeded("/apacheIgnite", null, CreateMode.PERSISTENT);
+
+        List<String> paths = new ArrayList<>();
+
+        paths.add("/apacheIgnite/1");
+        paths.add("/apacheIgnite/2");
+        paths.add("/apacheIgnite/3");
+
+        client.createAll(paths, CreateMode.PERSISTENT);
+
+        assertEquals(3, client.getChildren("/apacheIgnite").size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeleteAll() throws Exception {
+        startZK(1);
+
+        ZookeeperClient client = createClient(SES_TIMEOUT);
+
+        client.createIfNeeded("/apacheIgnite", null, CreateMode.PERSISTENT);
+        client.createIfNeeded("/apacheIgnite/1", null, CreateMode.PERSISTENT);
+        client.createIfNeeded("/apacheIgnite/2", null, CreateMode.PERSISTENT);
+
+        client.deleteAll("/apacheIgnite", Arrays.asList("1", "2"), -1);
+
+        assertTrue(client.getChildren("/apacheIgnite").isEmpty());
+
+        client.createIfNeeded("/apacheIgnite/1", null, CreateMode.PERSISTENT);
+        client.deleteAll("/apacheIgnite", Collections.singletonList("1"), -1);
+
+        assertTrue(client.getChildren("/apacheIgnite").isEmpty());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionLoss1() throws Exception {
+        ZookeeperClient client = new ZookeeperClient(log, "localhost:2200", 3000, null);
+
+        try {
+            client.createIfNeeded("/apacheIgnite", null, CreateMode.PERSISTENT);
+
+            fail();
+        }
+        catch (ZookeeperClientFailedException e) {
+            info("Expected error: " + e);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionLoss2() throws Exception {
+        startZK(1);
+
+        ZookeeperClient client = createClient(3000);
+
+        client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT);
+
+        closeZK();
+
+        try {
+            client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT);
+
+            fail();
+        }
+        catch (ZookeeperClientFailedException e) {
+            info("Expected error: " + e);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionLoss3() throws Exception {
+        startZK(1);
+
+        CallbackFuture cb = new CallbackFuture();
+
+        ZookeeperClient client = new ZookeeperClient(log, zkCluster.getConnectString(), 3000, cb);
+
+        client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT);
+
+        closeZK();
+
+        final AtomicBoolean res = new AtomicBoolean();
+
+        client.getChildrenAsync("/apacheIgnite1", null, new AsyncCallback.Children2Callback() {
+            @Override public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
+                if (rc == 0)
+                    res.set(true);
+            }
+        });
+
+        cb.get(60_000);
+
+        assertFalse(res.get());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionLoss4() throws Exception {
+        startZK(1);
+
+        CallbackFuture cb = new CallbackFuture();
+
+        final ZookeeperClient client = new ZookeeperClient(log, zkCluster.getConnectString(), 3000, cb);
+
+        client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT);
+
+        final CountDownLatch l = new CountDownLatch(1);
+
+        client.getChildrenAsync("/apacheIgnite1", null, new AsyncCallback.Children2Callback() {
+            @Override public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
+                closeZK();
+
+                try {
+                    client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT);
+                }
+                catch (ZookeeperClientFailedException e) {
+                    info("Expected error: " + e);
+
+                    l.countDown();
+                }
+                catch (Exception e) {
+                    fail("Unexpected error: " + e);
+                }
+            }
+        });
+
+        assertTrue(l.await(10, TimeUnit.SECONDS));
+
+        cb.get();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnect1() throws Exception {
+        startZK(1);
+
+        ZookeeperClient client = createClient(SES_TIMEOUT);
+
+        client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT);
+
+        zkCluster.getServers().get(0).stop();
+
+        IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                U.sleep(2000);
+
+                info("Restart zookeeper server");
+
+                zkCluster.getServers().get(0).restart();
+
+                info("Zookeeper server restarted");
+
+                return null;
+            }
+        }, "start-zk");
+
+        client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT);
+
+        fut.get();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnect1_Callback() throws Exception {
+        startZK(1);
+
+        ZookeeperClient client = createClient(SES_TIMEOUT);
+
+        client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT);
+
+        zkCluster.getServers().get(0).stop();
+
+        final CountDownLatch l = new CountDownLatch(1);
+
+        client.getChildrenAsync("/apacheIgnite1", null, new AsyncCallback.Children2Callback() {
+            @Override public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
+                info("Callback: " + rc);
+
+                if (rc == 0)
+                    l.countDown();
+            }
+        });
+
+        IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                U.sleep(2000);
+
+                info("Restart zookeeper server");
+
+                zkCluster.getServers().get(0).restart();
+
+                info("Zookeeper server restarted");
+
+                return null;
+            }
+        }, "start-zk");
+
+        assertTrue(l.await(10, TimeUnit.SECONDS));
+
+        fut.get();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnect1_InCallback() throws Exception {
+        startZK(1);
+
+        final ZookeeperClient client = createClient(SES_TIMEOUT);
+
+        client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT);
+
+        final CountDownLatch l = new CountDownLatch(1);
+
+        client.getChildrenAsync("/apacheIgnite1", null, new AsyncCallback.Children2Callback() {
+            @Override public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
+                try {
+                    zkCluster.getServers().get(0).stop();
+
+                    IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable<Void>() {
+                        @Override public Void call() throws Exception {
+                            U.sleep(2000);
+
+                            info("Restart zookeeper server");
+
+                            zkCluster.getServers().get(0).restart();
+
+                            info("Zookeeper server restarted");
+
+                            return null;
+                        }
+                    }, "start-zk");
+
+                    client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT);
+
+                    l.countDown();
+
+                    fut.get();
+                }
+                catch (Exception e) {
+                    fail("Unexpected error: " + e);
+                }
+            }
+        });
+
+        assertTrue(l.await(10, TimeUnit.SECONDS));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnect2() throws Exception {
+        startZK(1);
+
+        ZookeeperClient client = createClient(SES_TIMEOUT);
+
+        client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT);
+
+        zkCluster.getServers().get(0).restart();
+
+        client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnect3() throws Exception {
+        startZK(3);
+
+        ZookeeperClient client = createClient(SES_TIMEOUT);
+
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        for (int i = 0; i < 30; i++) {
+            info("Iteration: " + i);
+
+            int idx = rnd.nextInt(3);
+
+            zkCluster.getServers().get(idx).restart();
+
+            doSleep(rnd.nextLong(100) + 1);
+
+            client.createIfNeeded("/apacheIgnite" + i, null, CreateMode.PERSISTENT);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnect4() throws Exception {
+        startZK(3);
+
+        ZookeeperClient client = new ZookeeperClient(log,
+            zkCluster.getServers().get(2).getInstanceSpec().getConnectString(),
+            60_000,
+            null);
+
+        client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT);
+
+        zkCluster.getServers().get(0).stop();
+        zkCluster.getServers().get(1).stop();
+
+        IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                U.sleep(2000);
+
+                info("Restart zookeeper server");
+
+                zkCluster.getServers().get(0).restart();
+
+                info("Zookeeper server restarted");
+
+                return null;
+            }
+        }, "start-zk");
+
+        client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT);
+
+        fut.get();
+    }
+
+    /**
+     * @param instances Number of servers in ZK ensemble.
+     * @throws Exception If failed.
+     */
+    private void startZK(int instances) throws Exception {
+        assert zkCluster == null;
+
+        zkCluster = new TestingCluster(instances);
+
+        zkCluster.start();
+    }
+
+    /**
+     *
+     */
+    private void closeZK() {
+        if (zkCluster != null) {
+            try {
+                zkCluster.close();
+            }
+            catch (Exception e) {
+                U.error(log, "Failed to stop Zookeeper client: " + e, e);
+            }
+
+            zkCluster = null;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class CallbackFuture extends GridFutureAdapter<Void> implements IgniteRunnable {
+        /** {@inheritDoc} */
+        @Override public void run() {
+            onDone();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslAuthAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslAuthAbstractTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslAuthAbstractTest.java
new file mode 100644
index 0000000..ac94bf2
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslAuthAbstractTest.java
@@ -0,0 +1,247 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.file.Paths;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.zookeeper.client.ZooKeeperSaslClient;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.apache.zookeeper.server.ZKDatabase;
+import org.apache.zookeeper.server.ZooKeeperServer;
+
+import static org.apache.curator.test.DirectoryUtils.deleteRecursively;
+
+/**
+ * Implements methods to prepare SASL tests infrastructure: jaas.conf files, starting up ZooKeeper server,
+ * clean up procedures when the test has finished etc.
+ */
+public abstract class ZookeeperDiscoverySpiSaslAuthAbstractTest extends GridCommonAbstractTest {
+    /** */
+    private File tmpDir = createTmpDir();
+
+    /** */
+    private static final String JAAS_CONF_FILE = "jaas.conf";
+
+    /** */
+    private static final String AUTH_PROVIDER = "zookeeper.authProvider.1";
+
+    /** */
+    private static final String SASL_CONFIG = "java.security.auth.login.config";
+
+    /** */
+    private long joinTimeout = 2_000;
+
+    /** */
+    private long sesTimeout = 10_000;
+
+    /** */
+    private ServerCnxnFactory serverFactory;
+
+    /** */
+    private String hostPort = "localhost:2181";
+
+    /** */
+    private int maxCnxns;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String instanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(instanceName);
+
+        ZookeeperDiscoverySpi zkSpi = new ZookeeperDiscoverySpi();
+
+        if (joinTimeout != 0)
+            zkSpi.setJoinTimeout(joinTimeout);
+
+        zkSpi.setSessionTimeout(sesTimeout > 0 ? sesTimeout : 10_000);
+
+        zkSpi.setZkConnectionString(hostPort);
+
+        cfg.setDiscoverySpi(zkSpi);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        prepareJaasConfigFile();
+
+        prepareSaslSystemProperties();
+
+        startZooKeeperServer();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopZooKeeperServer();
+
+        stopAllGrids();
+
+        clearSaslSystemProperties();
+
+        clearTmpDir();
+    }
+
+    /** */
+    private void clearTmpDir() throws Exception {
+        deleteRecursively(tmpDir);
+    }
+
+    /** */
+    protected void clearSaslSystemProperties() {
+        System.clearProperty(AUTH_PROVIDER);
+
+        System.clearProperty(SASL_CONFIG);
+
+        System.clearProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void prepareJaasConfigFile() throws Exception {
+        U.ensureDirectory(tmpDir, "Temp directory for JAAS configuration file wasn't created", null);
+
+        File saslConfFile = new File(tmpDir, JAAS_CONF_FILE);
+
+        FileWriter fwriter = new FileWriter(saslConfFile);
+
+        writeServerConfigSection(fwriter, "validPassword");
+
+        writeClientConfigSection(fwriter, "ValidZookeeperClient", "validPassword");
+
+        writeClientConfigSection(fwriter, "InvalidZookeeperClient", "invalidPassword");
+
+        fwriter.close();
+    }
+
+    /** */
+    private void prepareSaslSystemProperties() {
+        System.setProperty(SASL_CONFIG, Paths.get(tmpDir.getPath().toString(), JAAS_CONF_FILE).toString());
+
+        System.setProperty(AUTH_PROVIDER, "org.apache.zookeeper.server.auth.SASLAuthenticationProvider");
+    }
+
+    /** */
+    private void writeClientConfigSection(FileWriter fwriter, String clientName, String pass) throws IOException {
+        fwriter.write(clientName + "{\n" +
+            "       org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
+            "       username=\"zkUser\"\n" +
+            "       password=\"" + pass + "\";\n" +
+            "};" + "\n");
+    }
+
+    /** */
+    private void writeServerConfigSection(FileWriter fwriter, String pass) throws IOException {
+        fwriter.write("Server {\n" +
+            "          org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
+            "          user_zkUser=\"" + pass + "\";\n" +
+            "};\n");
+    }
+
+    /** */
+    private File createTmpDir() {
+        File jaasConfDir = Paths.get(System.getProperty("java.io.tmpdir"), "zk_disco_spi_test").toFile();
+
+        try {
+            U.ensureDirectory(jaasConfDir, "", null);
+        }
+        catch (IgniteCheckedException e) {
+            // ignored
+        }
+
+        return jaasConfDir;
+    }
+
+    /** */
+    private void stopZooKeeperServer() throws Exception {
+        shutdownServerInstance(serverFactory);
+        serverFactory = null;
+    }
+
+    /** */
+    private void shutdownServerInstance(ServerCnxnFactory factory)
+    {
+        if (factory != null) {
+            ZKDatabase zkDb = null;
+            {
+                ZooKeeperServer zs = getServer(factory);
+                if (zs != null)
+                    zkDb = zs.getZKDatabase();
+            }
+            factory.shutdown();
+            try {
+                if (zkDb != null)
+                    zkDb.close();
+            } catch (IOException ie) {
+                // ignore
+            }
+        }
+    }
+
+    /** */
+    private ZooKeeperServer getServer(ServerCnxnFactory fac) {
+        ZooKeeperServer zs = U.field(fac, "zkServer");
+
+        return zs;
+    }
+
+    /** */
+    private void startZooKeeperServer() throws Exception {
+        serverFactory = createNewServerInstance(serverFactory, hostPort,
+            maxCnxns);
+        startServerInstance(tmpDir, serverFactory);
+    }
+
+    /** */
+    private ServerCnxnFactory createNewServerInstance(
+        ServerCnxnFactory factory, String hostPort, int maxCnxns)
+        throws IOException {
+        final int port = getPort(hostPort);
+
+        if (factory == null)
+            factory = ServerCnxnFactory.createFactory(port, maxCnxns);
+
+        return factory;
+    }
+
+    /** */
+    private void startServerInstance(File dataDir,
+        ServerCnxnFactory factory) throws IOException,
+        InterruptedException {
+        ZooKeeperServer zks = new ZooKeeperServer(dataDir, dataDir, 3000);
+        factory.startup(zks);
+    }
+
+    /** */
+    private int getPort(String hostPort) {
+        String[] split = hostPort.split(":");
+        String portstr = split[split.length-1];
+        String[] pc = portstr.split("/");
+
+        if (pc.length > 1)
+            portstr = pc[0];
+
+        return Integer.parseInt(portstr);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslFailedAuthTest.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslFailedAuthTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslFailedAuthTest.java
new file mode 100644
index 0000000..864ac96
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslFailedAuthTest.java
@@ -0,0 +1,44 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import org.apache.zookeeper.client.ZooKeeperSaslClient;
+import org.junit.Assert;
+
+/**
+ *
+ */
+public class ZookeeperDiscoverySpiSaslFailedAuthTest extends ZookeeperDiscoverySpiSaslAuthAbstractTest {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIgniteNodeWithInvalidPasswordFailsToJoin() throws Exception {
+        System.setProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY,
+            "InvalidZookeeperClient");
+
+        System.setProperty("IGNITE_ZOOKEEPER_DISCOVERY_MAX_RETRY_COUNT", Integer.toString(1));
+
+        try {
+            startGrid(0);
+
+            Assert.fail("Ignite node with invalid password should fail on join.");
+        }
+        catch (Exception e) {
+            //ignored
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslSuccessfulAuthTest.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslSuccessfulAuthTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslSuccessfulAuthTest.java
new file mode 100644
index 0000000..5ee0a43
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslSuccessfulAuthTest.java
@@ -0,0 +1,48 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import org.apache.zookeeper.client.ZooKeeperSaslClient;
+
+/**
+ *
+ */
+public class ZookeeperDiscoverySpiSaslSuccessfulAuthTest extends ZookeeperDiscoverySpiSaslAuthAbstractTest {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIgniteNodesWithValidPasswordSuccessfullyJoins() throws Exception {
+        System.setProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY,
+            "ValidZookeeperClient");
+
+        startGrids(3);
+
+        waitForTopology(3);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIgniteNodeWithoutSaslConfigurationSuccessfullyJoins() throws Exception {
+        //clearing SASL-related system properties that were set in beforeTest
+        clearSaslSystemProperties();
+
+        startGrid(0);
+
+        waitForTopology(1);
+    }
+}


[05/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClient.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClient.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClient.java
new file mode 100644
index 0000000..21703c6
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClient.java
@@ -0,0 +1,1219 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Zookeeper Client.
+ */
+public class ZookeeperClient implements Watcher {
+    /** */
+    private static final long RETRY_TIMEOUT =
+        IgniteSystemProperties.getLong("IGNITE_ZOOKEEPER_DISCOVERY_RETRY_TIMEOUT", 2000);
+
+    /** */
+    private static final int MAX_RETRY_COUNT =
+        IgniteSystemProperties.getInteger("IGNITE_ZOOKEEPER_DISCOVERY_MAX_RETRY_COUNT", 10);
+
+    /** */
+    private final AtomicInteger retryCount = new AtomicInteger();
+
+    /** */
+    private static final int MAX_REQ_SIZE = 1048528;
+
+    /** */
+    private static final List<ACL> ZK_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE;
+
+    /** */
+    private static final byte[] EMPTY_BYTES = {};
+
+    /** */
+    private final ZooKeeper zk;
+
+    /** */
+    private final IgniteLogger log;
+
+    /** */
+    private ConnectionState state = ConnectionState.Disconnected;
+
+    /** */
+    private long connLossTimeout;
+
+    /** */
+    private volatile long connStartTime;
+
+    /** */
+    private final Object stateMux = new Object();
+
+    /** */
+    private final IgniteRunnable connLostC;
+
+    /** */
+    private final Timer connTimer;
+
+    /** */
+    private final ArrayDeque<ZkAsyncOperation> retryQ = new ArrayDeque<>();
+
+    /** */
+    private volatile boolean closing;
+
+    /**
+     * @param log Logger.
+     * @param connectString ZK connection string.
+     * @param sesTimeout ZK session timeout.
+     * @param connLostC Lost connection callback.
+     * @throws Exception If failed.
+     */
+    ZookeeperClient(IgniteLogger log, String connectString, int sesTimeout, IgniteRunnable connLostC) throws Exception {
+        this(null, log, connectString, sesTimeout, connLostC);
+    }
+
+    /**
+     * @param igniteInstanceName Ignite instance name.
+     * @param log Logger.
+     * @param connectString ZK connection string.
+     * @param sesTimeout ZK session timeout.
+     * @param connLostC Lost connection callback.
+     * @throws Exception If failed.
+     */
+    ZookeeperClient(String igniteInstanceName,
+        IgniteLogger log,
+        String connectString,
+        int sesTimeout,
+        IgniteRunnable connLostC)
+        throws Exception
+    {
+        this.log = log.getLogger(getClass());
+        this.connLostC = connLostC;
+
+        connLossTimeout = sesTimeout;
+
+        long connStartTime = this.connStartTime = System.currentTimeMillis();
+
+        connTimer = new Timer("zk-client-timer-" + igniteInstanceName);
+
+        String threadName = Thread.currentThread().getName();
+
+        // ZK generates internal threads' names using current thread name.
+        Thread.currentThread().setName("zk-" + igniteInstanceName);
+
+        try {
+            zk = new ZooKeeper(connectString, sesTimeout, this);
+        }
+        finally {
+            Thread.currentThread().setName(threadName);
+        }
+
+        synchronized (stateMux) {
+            if (connStartTime == this.connStartTime && state == ConnectionState.Disconnected)
+                scheduleConnectionCheck();
+        }
+    }
+
+    /**
+     * @return Zookeeper client.
+     */
+    ZooKeeper zk() {
+        return zk;
+    }
+
+    /**
+     * @return {@code True} if connected to ZooKeeper.
+     */
+    boolean connected() {
+        synchronized (stateMux) {
+            return state == ConnectionState.Connected;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void process(WatchedEvent evt) {
+        if (closing)
+            return;
+
+        if (evt.getType() == Event.EventType.None) {
+            ConnectionState newState;
+
+            synchronized (stateMux) {
+                if (state == ConnectionState.Lost) {
+                    U.warn(log, "Received event after connection was lost [evtState=" + evt.getState() + "]");
+
+                    return;
+                }
+
+                if (!zk.getState().isAlive())
+                    return;
+
+                Event.KeeperState zkState = evt.getState();
+
+                switch (zkState) {
+                    case SaslAuthenticated:
+                        return; // No-op.
+
+                    case AuthFailed:
+                        newState = state;
+
+                        break;
+
+                    case Disconnected:
+                        newState = ConnectionState.Disconnected;
+
+                        break;
+
+                    case SyncConnected:
+                        newState = ConnectionState.Connected;
+
+                        break;
+
+                    case Expired:
+                        U.warn(log, "Session expired, changing state to Lost");
+
+                        newState = ConnectionState.Lost;
+
+                        break;
+
+                    default:
+                        U.error(log, "Unexpected state for ZooKeeper client, close connection: " + zkState);
+
+                        newState = ConnectionState.Lost;
+                }
+
+                if (newState != state) {
+                    if (log.isInfoEnabled())
+                        log.info("ZooKeeper client state changed [prevState=" + state + ", newState=" + newState + ']');
+
+                    state = newState;
+
+                    if (newState == ConnectionState.Disconnected) {
+                        connStartTime = System.currentTimeMillis();
+
+                        scheduleConnectionCheck();
+                    }
+                    else if (newState == ConnectionState.Connected) {
+                        retryCount.set(0);
+
+                        stateMux.notifyAll();
+                    }
+                    else
+                        assert state == ConnectionState.Lost : state;
+                }
+                else
+                    return;
+            }
+
+            if (newState == ConnectionState.Lost) {
+                closeClient();
+
+                notifyConnectionLost();
+            }
+            else if (newState == ConnectionState.Connected) {
+                for (ZkAsyncOperation op : retryQ)
+                    op.execute();
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    private void notifyConnectionLost() {
+        if (!closing && state == ConnectionState.Lost && connLostC != null)
+            connLostC.run();
+
+        connTimer.cancel();
+    }
+
+    /**
+     * @param path Path.
+     * @return {@code True} if node exists.
+     * @throws ZookeeperClientFailedException If connection to zk was lost.
+     * @throws InterruptedException If interrupted.
+     */
+    boolean exists(String path) throws ZookeeperClientFailedException, InterruptedException {
+        for (;;) {
+            long connStartTime = this.connStartTime;
+
+            try {
+                return zk.exists(path, false) != null;
+            }
+            catch (Exception e) {
+                onZookeeperError(connStartTime, e);
+            }
+        }
+    }
+
+    /**
+     *
+     * @param paths Paths to create.
+     * @param createMode Create mode.
+     * @throws KeeperException.NodeExistsException If at least one of target node already exists.
+     * @throws ZookeeperClientFailedException If connection to zk was lost.
+     * @throws InterruptedException If interrupted.
+     */
+    void createAll(List<String> paths, CreateMode createMode)
+        throws ZookeeperClientFailedException, InterruptedException, KeeperException.NodeExistsException
+    {
+        // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8188
+        List<Op> ops = new ArrayList<>(paths.size());
+
+        for (String path : paths)
+            ops.add(Op.create(path, EMPTY_BYTES, ZK_ACL, createMode));
+
+        for (;;) {
+            long connStartTime = this.connStartTime;
+
+            try {
+                zk.multi(ops);
+
+                return;
+            }
+            catch (KeeperException.NodeExistsException e) {
+                throw e;
+            }
+            catch (Exception e) {
+                onZookeeperError(connStartTime, e);
+            }
+        }
+    }
+
+    /**
+     * @param path Path.
+     * @param data Data.
+     * @param overhead Extra overhead.
+     * @return {@code True} If data size exceeds max request size and should be splitted into multiple parts.
+     */
+    boolean needSplitNodeData(String path, byte[] data, int overhead) {
+        return requestOverhead(path) + data.length + overhead > MAX_REQ_SIZE;
+    }
+
+    /**
+     * @param path Path.
+     * @param data Data.
+     * @param overhead Extra overhead.
+     * @return Splitted data.
+     */
+    List<byte[]> splitNodeData(String path, byte[] data, int overhead) {
+        int partSize = MAX_REQ_SIZE - requestOverhead(path) - overhead;
+
+        int partCnt = data.length / partSize;
+
+        if (data.length % partSize != 0)
+            partCnt++;
+
+        assert partCnt > 1 : "Do not need split";
+
+        List<byte[]> parts = new ArrayList<>(partCnt);
+
+        int remaining = data.length;
+
+        for (int i = 0; i < partCnt; i++) {
+            int partSize0 = Math.min(remaining, partSize);
+
+            byte[] part = new byte[partSize0];
+
+            System.arraycopy(data, i * partSize, part, 0, part.length);
+
+            remaining -= partSize0;
+
+            parts.add(part);
+        }
+
+        assert remaining == 0 : remaining;
+
+        return parts;
+    }
+
+    /**
+     * TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8187
+     * @param path Request path.
+     * @return Marshalled request overhead.
+     */
+    private int requestOverhead(String path) {
+        return path.length();
+    }
+
+    /**
+     * @param path Path.
+     * @param data Data.
+     * @param createMode Create mode.
+     * @return Created path.
+     * @throws ZookeeperClientFailedException If connection to zk was lost.
+     * @throws InterruptedException If interrupted.
+     */
+    String createIfNeeded(String path, byte[] data, CreateMode createMode)
+        throws ZookeeperClientFailedException, InterruptedException
+    {
+        assert !createMode.isSequential() : createMode;
+
+        if (data == null)
+            data = EMPTY_BYTES;
+
+        for (;;) {
+            long connStartTime = this.connStartTime;
+
+            try {
+                return zk.create(path, data, ZK_ACL, createMode);
+            }
+            catch (KeeperException.NodeExistsException e) {
+                if (log.isDebugEnabled())
+                    log.debug("Node already exists: " + path);
+
+                return path;
+            }
+            catch (Exception e) {
+                onZookeeperError(connStartTime, e);
+            }
+        }
+    }
+
+    /**
+     * @param checkPrefix Unique prefix to check in case of retry.
+     * @param parentPath Parent node path.
+     * @param path Node to create.
+     * @param data Node data.
+     * @param createMode Create mode.
+     * @return Create path.
+     * @throws ZookeeperClientFailedException If connection to zk was lost.
+     * @throws InterruptedException If interrupted.
+     */
+    String createSequential(String checkPrefix, String parentPath, String path, byte[] data, CreateMode createMode)
+        throws ZookeeperClientFailedException, InterruptedException
+    {
+        assert createMode.isSequential() : createMode;
+
+        if (data == null)
+            data = EMPTY_BYTES;
+
+        boolean first = true;
+
+        for (;;) {
+            long connStartTime = this.connStartTime;
+
+            try {
+                if (!first) {
+                    List<String> children = zk.getChildren(parentPath, false);
+
+                    for (int i = 0; i < children.size(); i++) {
+                        String child = children.get(i);
+
+                        if (children.get(i).startsWith(checkPrefix)) {
+                            String resPath = parentPath + "/" + child;
+
+                            if (log.isDebugEnabled())
+                                log.debug("Check before retry, node already created: " + resPath);
+
+                            return resPath;
+                        }
+                    }
+                }
+
+                return zk.create(path, data, ZK_ACL, createMode);
+            }
+            catch (KeeperException.NodeExistsException e) {
+                assert !createMode.isSequential() : createMode;
+
+                if (log.isDebugEnabled())
+                    log.debug("Node already exists: " + path);
+
+                return path;
+            }
+            catch (Exception e) {
+                onZookeeperError(connStartTime, e);
+            }
+
+            first = false;
+        }
+    }
+
+    /**
+     * @param path Path.
+     * @return Children nodes.
+     * @throws ZookeeperClientFailedException If connection to zk was lost.
+     * @throws InterruptedException If interrupted.
+     */
+    List<String> getChildren(String path)
+        throws ZookeeperClientFailedException, InterruptedException
+    {
+        for (;;) {
+            long connStartTime = this.connStartTime;
+
+            try {
+                return zk.getChildren(path, false);
+            }
+            catch (Exception e) {
+                onZookeeperError(connStartTime, e);
+            }
+        }
+    }
+
+    /**
+     * @param path Path.
+     * @throws InterruptedException If interrupted.
+     * @throws KeeperException In case of error.
+     * @return {@code True} if given path exists.
+     */
+    boolean existsNoRetry(String path) throws InterruptedException, KeeperException {
+        return zk.exists(path, false) != null;
+    }
+
+    /**
+     * @param path Path.
+     * @param ver Expected version.
+     * @throws InterruptedException If interrupted.
+     * @throws KeeperException In case of error.
+     */
+    void deleteIfExistsNoRetry(String path, int ver) throws InterruptedException, KeeperException {
+        try {
+            zk.delete(path, ver);
+        }
+        catch (KeeperException.NoNodeException e) {
+            // No-op if znode does not exist.
+        }
+    }
+
+    /**
+     * @param path Path.
+     * @param ver Version.
+     * @throws ZookeeperClientFailedException If connection to zk was lost.
+     * @throws InterruptedException If interrupted.
+     */
+    void deleteIfExists(String path, int ver)
+        throws ZookeeperClientFailedException, InterruptedException
+    {
+        try {
+            delete(path, ver);
+        }
+        catch (KeeperException.NoNodeException e) {
+            // No-op if znode does not exist.
+        }
+    }
+
+    /**
+     * @param parent Parent path.
+     * @param paths Children paths.
+     * @param ver Version.
+     * @throws KeeperException.NoNodeException If at least one of nodes does not exist.
+     * @throws ZookeeperClientFailedException If connection to zk was lost.
+     * @throws InterruptedException If interrupted.
+     */
+    void deleteAll(@Nullable String parent, List<String> paths, int ver)
+        throws KeeperException.NoNodeException, ZookeeperClientFailedException, InterruptedException
+    {
+        if (paths.isEmpty())
+            return;
+
+        // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8188
+        List<Op> ops = new ArrayList<>(paths.size());
+
+        for (String path : paths) {
+            String path0 = parent != null ? parent + "/" + path : path;
+
+            ops.add(Op.delete(path0, ver));
+        }
+
+        for (;;) {
+            long connStartTime = this.connStartTime;
+
+            try {
+                zk.multi(ops);
+
+                return;
+            }
+            catch (KeeperException.NoNodeException e) {
+                throw e;
+            }
+            catch (Exception e) {
+                onZookeeperError(connStartTime, e);
+            }
+        }
+    }
+
+    /**
+     * @param path Path.
+     * @param ver Version.
+     * @throws KeeperException.NoNodeException If target node does not exist.
+     * @throws ZookeeperClientFailedException If connection to zk was lost.
+     * @throws InterruptedException If interrupted.
+     */
+    private void delete(String path, int ver)
+        throws KeeperException.NoNodeException, ZookeeperClientFailedException, InterruptedException
+    {
+        for (;;) {
+            long connStartTime = this.connStartTime;
+
+            try {
+                zk.delete(path, ver);
+
+                return;
+            }
+            catch (KeeperException.NoNodeException e) {
+                throw e;
+            }
+            catch (Exception e) {
+                onZookeeperError(connStartTime, e);
+            }
+        }
+    }
+
+    /**
+     * @param path Path.
+     * @param data Data.
+     * @param ver Version.
+     * @throws ZookeeperClientFailedException If connection to zk was lost.
+     * @throws InterruptedException If interrupted.
+     * @throws KeeperException.NoNodeException If node does not exist.
+     * @throws KeeperException.BadVersionException If version does not match.
+     */
+    void setData(String path, byte[] data, int ver)
+        throws ZookeeperClientFailedException, InterruptedException, KeeperException.NoNodeException,
+        KeeperException.BadVersionException
+    {
+        if (data == null)
+            data = EMPTY_BYTES;
+
+        for (;;) {
+            long connStartTime = this.connStartTime;
+
+            try {
+                zk.setData(path, data, ver);
+
+                return;
+            }
+            catch (KeeperException.BadVersionException | KeeperException.NoNodeException e) {
+                throw e;
+            }
+            catch (Exception e) {
+                onZookeeperError(connStartTime, e);
+            }
+        }
+    }
+
+    /**
+     * @param path Path.
+     * @param stat Optional {@link Stat} instance to return znode state.
+     * @return Data.
+     * @throws KeeperException.NoNodeException If target node does not exist.
+     * @throws ZookeeperClientFailedException If connection to zk was lost.
+     * @throws InterruptedException If interrupted.
+     */
+    byte[] getData(String path, @Nullable Stat stat)
+        throws KeeperException.NoNodeException, ZookeeperClientFailedException, InterruptedException {
+        for (;;) {
+            long connStartTime = this.connStartTime;
+
+            try {
+                return zk.getData(path, false, stat);
+            }
+            catch (KeeperException.NoNodeException e) {
+                throw e;
+            }
+            catch (Exception e) {
+                onZookeeperError(connStartTime, e);
+            }
+        }
+    }
+
+    /**
+     * @param path Path.
+     * @return Data.
+     * @throws KeeperException.NoNodeException If target node does not exist.
+     * @throws ZookeeperClientFailedException If connection to zk was lost.
+     * @throws InterruptedException If interrupted.
+     */
+    byte[] getData(String path)
+        throws KeeperException.NoNodeException, ZookeeperClientFailedException, InterruptedException
+    {
+        return getData(path, null);
+    }
+
+    /**
+     * @param path Path.
+     */
+    void deleteIfExistsAsync(String path) {
+        new DeleteIfExistsOperation(path).execute();
+    }
+
+    /**
+     * @param path Path.
+     * @param watcher Watcher.
+     * @param cb Callback.
+     */
+    void existsAsync(String path, Watcher watcher, AsyncCallback.StatCallback cb) {
+        ExistsOperation op = new ExistsOperation(path, watcher, cb);
+
+        zk.exists(path, watcher, new StatCallbackWrapper(op), null);
+    }
+
+    /**
+     * @param path Path.
+     * @param watcher Watcher.
+     * @param cb Callback.
+     */
+    void getChildrenAsync(String path, Watcher watcher, AsyncCallback.Children2Callback cb) {
+        GetChildrenOperation op = new GetChildrenOperation(path, watcher, cb);
+
+        zk.getChildren(path, watcher, new ChildrenCallbackWrapper(op), null);
+    }
+
+    /**
+     * @param path Path.
+     * @param watcher Watcher.
+     * @param cb Callback.
+     */
+    void getDataAsync(String path, Watcher watcher, AsyncCallback.DataCallback cb) {
+        GetDataOperation op = new GetDataOperation(path, watcher, cb);
+
+        zk.getData(path, watcher, new DataCallbackWrapper(op), null);
+    }
+
+    /**
+     * @param path Path.
+     * @param data Data.
+     * @param createMode Create mode.
+     * @param cb Callback.
+     */
+    private void createAsync(String path, byte[] data, CreateMode createMode, AsyncCallback.StringCallback cb) {
+        if (data == null)
+            data = EMPTY_BYTES;
+
+        CreateOperation op = new CreateOperation(path, data, createMode, cb);
+
+        zk.create(path, data, ZK_ACL, createMode, new CreateCallbackWrapper(op), null);
+    }
+
+    /**
+     *
+     */
+    void onCloseStart() {
+        closing = true;
+
+        synchronized (stateMux) {
+            stateMux.notifyAll();
+        }
+    }
+
+    /**
+     *
+     */
+    public void close() {
+        closeClient();
+    }
+
+    /**
+     * @param prevConnStartTime Time when connection was established.
+     * @param e Error.
+     * @throws ZookeeperClientFailedException If connection to zk was lost.
+     * @throws InterruptedException If interrupted.
+     */
+    private void onZookeeperError(long prevConnStartTime, Exception e)
+        throws ZookeeperClientFailedException, InterruptedException
+    {
+        ZookeeperClientFailedException err = null;
+
+        synchronized (stateMux) {
+            if (closing)
+                throw new ZookeeperClientFailedException("ZooKeeper client is closed.");
+
+            U.warn(log, "Failed to execute ZooKeeper operation [err=" + e + ", state=" + state + ']');
+
+            if (state == ConnectionState.Lost) {
+                U.error(log, "Operation failed with unexpected error, connection lost: " + e, e);
+
+                throw new ZookeeperClientFailedException(e);
+            }
+
+            boolean retry = (e instanceof KeeperException) && needRetry(((KeeperException)e).code().intValue());
+
+            if (retry) {
+                long remainingTime;
+
+                if (state == ConnectionState.Connected && connStartTime == prevConnStartTime) {
+                    state = ConnectionState.Disconnected;
+
+                    connStartTime = System.currentTimeMillis();
+
+                    remainingTime = connLossTimeout;
+                }
+                else {
+                    assert connStartTime != 0;
+
+                    assert state == ConnectionState.Disconnected : state;
+
+                    remainingTime = connLossTimeout - (System.currentTimeMillis() - connStartTime);
+
+                    if (remainingTime <= 0) {
+                        state = ConnectionState.Lost;
+
+                        U.warn(log, "Failed to establish ZooKeeper connection, close client " +
+                            "[timeout=" + connLossTimeout + ']');
+
+                        err = new ZookeeperClientFailedException(e);
+                    }
+                }
+
+                if (err == null) {
+                    U.warn(log, "ZooKeeper operation failed, will retry [err=" + e +
+                        ", retryTimeout=" + RETRY_TIMEOUT +
+                        ", connLossTimeout=" + connLossTimeout +
+                        ", path=" + ((KeeperException)e).getPath() +
+                        ", remainingWaitTime=" + remainingTime + ']');
+
+                    stateMux.wait(RETRY_TIMEOUT);
+
+                    if (closing)
+                        throw new ZookeeperClientFailedException("ZooKeeper client is closed.");
+                }
+            }
+            else {
+                U.error(log, "Operation failed with unexpected error, close ZooKeeper client: " + e, e);
+
+                state = ConnectionState.Lost;
+
+                err = new ZookeeperClientFailedException(e);
+            }
+        }
+
+        if (err != null) {
+            closeClient();
+
+            notifyConnectionLost();
+
+            throw err;
+        }
+    }
+
+    /**
+     * @param code Zookeeper error code.
+     * @return {@code True} if can retry operation.
+     */
+    private boolean needRetry(int code) {
+        boolean retryByErrorCode = code == KeeperException.Code.CONNECTIONLOSS.intValue() ||
+            code == KeeperException.Code.SESSIONMOVED.intValue() ||
+            code == KeeperException.Code.OPERATIONTIMEOUT.intValue();
+
+        if (retryByErrorCode) {
+            if (MAX_RETRY_COUNT <= 0 || retryCount.incrementAndGet() < MAX_RETRY_COUNT)
+                return true;
+            else
+                return false;
+        }
+        else
+            return false;
+    }
+
+    /**
+     *
+     */
+    private void closeClient() {
+        try {
+            zk.close();
+        }
+        catch (Exception closeErr) {
+            U.warn(log, "Failed to close ZooKeeper client: " + closeErr, closeErr);
+        }
+
+        connTimer.cancel();
+    }
+
+    /**
+     *
+     */
+    private void scheduleConnectionCheck() {
+        assert state == ConnectionState.Disconnected : state;
+
+        connTimer.schedule(new ConnectionTimeoutTask(connStartTime), connLossTimeout);
+    }
+
+    /**
+     *
+     */
+    interface ZkAsyncOperation {
+        /**
+         *
+         */
+        void execute();
+    }
+
+    /**
+     *
+     */
+    class GetChildrenOperation implements ZkAsyncOperation {
+        /** */
+        private final String path;
+
+        /** */
+        private final Watcher watcher;
+
+        /** */
+        private final AsyncCallback.Children2Callback cb;
+
+        /**
+         * @param path Path.
+         * @param watcher Watcher.
+         * @param cb Callback.
+         */
+        GetChildrenOperation(String path, Watcher watcher, AsyncCallback.Children2Callback cb) {
+            this.path = path;
+            this.watcher = watcher;
+            this.cb = cb;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void execute() {
+            getChildrenAsync(path, watcher, cb);
+        }
+    }
+
+    /**
+     *
+     */
+    class GetDataOperation implements ZkAsyncOperation {
+        /** */
+        private final String path;
+
+        /** */
+        private final Watcher watcher;
+
+        /** */
+        private final AsyncCallback.DataCallback cb;
+
+        /**
+         * @param path Path.
+         * @param watcher Watcher.
+         * @param cb Callback.
+         */
+        GetDataOperation(String path, Watcher watcher, AsyncCallback.DataCallback cb) {
+            this.path = path;
+            this.watcher = watcher;
+            this.cb = cb;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void execute() {
+            getDataAsync(path, watcher, cb);
+        }
+    }
+
+    /**
+     *
+     */
+    class ExistsOperation implements ZkAsyncOperation {
+        /** */
+        private final String path;
+
+        /** */
+        private final Watcher watcher;
+
+        /** */
+        private final AsyncCallback.StatCallback cb;
+
+        /**
+         * @param path Path.
+         * @param watcher Watcher.
+         * @param cb Callback.
+         */
+        ExistsOperation(String path, Watcher watcher, AsyncCallback.StatCallback cb) {
+            this.path = path;
+            this.watcher = watcher;
+            this.cb = cb;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void execute() {
+            existsAsync(path, watcher, cb);
+        }
+    }
+
+    /**
+     *
+     */
+    class CreateOperation implements ZkAsyncOperation {
+        /** */
+        private final String path;
+
+        /** */
+        private final byte[] data;
+
+        /** */
+        private final CreateMode createMode;
+
+        /** */
+        private final AsyncCallback.StringCallback cb;
+
+        /**
+         * @param path path.
+         * @param data Data.
+         * @param createMode Create mode.
+         * @param cb Callback.
+         */
+        CreateOperation(String path, byte[] data, CreateMode createMode, AsyncCallback.StringCallback cb) {
+            this.path = path;
+            this.data = data;
+            this.createMode = createMode;
+            this.cb = cb;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void execute() {
+            createAsync(path, data, createMode, cb);
+        }
+    }
+
+    /**
+     *
+     */
+    class DeleteIfExistsOperation implements AsyncCallback.VoidCallback, ZkAsyncOperation {
+        /** */
+        private final String path;
+
+        /**
+         * @param path Path.
+         */
+        DeleteIfExistsOperation(String path) {
+            this.path = path;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void execute() {
+            zk.delete(path, -1, this, null);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void processResult(int rc, String path, Object ctx) {
+            if (closing)
+                return;
+
+            if (rc == KeeperException.Code.NONODE.intValue())
+                return;
+
+            if (needRetry(rc)) {
+                U.warn(log, "Failed to execute async operation, connection lost. Will retry after connection restore [" +
+                    "path=" + path + ']');
+
+                retryQ.add(this);
+            }
+            else if (rc == KeeperException.Code.SESSIONEXPIRED.intValue())
+                U.warn(log, "Failed to execute async operation, connection lost [path=" + path + ']');
+            else
+                assert rc == 0 : KeeperException.Code.get(rc);
+        }
+    }
+
+    /**
+     *
+     */
+    class CreateCallbackWrapper implements AsyncCallback.StringCallback {
+        /** */
+        final CreateOperation op;
+
+        /**
+         * @param op Operation.
+         */
+        CreateCallbackWrapper(CreateOperation op) {
+            this.op = op;
+        }
+
+        @Override public void processResult(int rc, String path, Object ctx, String name) {
+            if (closing)
+                return;
+
+            if (rc == KeeperException.Code.NODEEXISTS.intValue())
+                return;
+
+            if (needRetry(rc)) {
+                U.warn(log, "Failed to execute async operation, connection lost. Will retry after connection restore [path=" + path + ']');
+
+                retryQ.add(op);
+            }
+            else if (rc == KeeperException.Code.SESSIONEXPIRED.intValue())
+                U.warn(log, "Failed to execute async operation, connection lost [path=" + path + ']');
+            else {
+                if (op.cb != null)
+                    op.cb.processResult(rc, path, ctx, name);
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    class ChildrenCallbackWrapper implements AsyncCallback.Children2Callback {
+        /** */
+        private final GetChildrenOperation op;
+
+        /**
+         * @param op Operation.
+         */
+        private ChildrenCallbackWrapper(GetChildrenOperation op) {
+            this.op = op;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
+            if (closing)
+                return;
+
+            if (needRetry(rc)) {
+                U.warn(log, "Failed to execute async operation, connection lost. Will retry after connection restore [path=" + path + ']');
+
+                retryQ.add(op);
+            }
+            else if (rc == KeeperException.Code.SESSIONEXPIRED.intValue())
+                U.warn(log, "Failed to execute async operation, connection lost [path=" + path + ']');
+            else
+                op.cb.processResult(rc, path, ctx, children, stat);
+        }
+    }
+
+    /**
+     *
+     */
+    class DataCallbackWrapper implements AsyncCallback.DataCallback {
+        /** */
+        private final GetDataOperation op;
+
+        /**
+         * @param op Operation.
+         */
+        private DataCallbackWrapper(GetDataOperation op) {
+            this.op = op;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
+            if (closing)
+                return;
+
+            if (needRetry(rc)) {
+                U.warn(log, "Failed to execute async operation, connection lost. Will retry after connection restore [path=" + path + ']');
+
+                retryQ.add(op);
+            }
+            else if (rc == KeeperException.Code.SESSIONEXPIRED.intValue())
+                U.warn(log, "Failed to execute async operation, connection lost [path=" + path + ']');
+            else
+                op.cb.processResult(rc, path, ctx, data, stat);
+        }
+    }
+
+    /**
+     *
+     */
+    class StatCallbackWrapper implements AsyncCallback.StatCallback {
+        /** */
+        private final ExistsOperation op;
+
+        /**
+         * @param op Operation.
+         */
+        private StatCallbackWrapper(ExistsOperation op) {
+            this.op = op;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void processResult(int rc, String path, Object ctx, Stat stat) {
+            if (closing)
+                return;
+
+            if (needRetry(rc)) {
+                U.warn(log, "Failed to execute async operation, connection lost. Will retry after connection restore [path=" + path + ']');
+
+                retryQ.add(op);
+            }
+            else if (rc == KeeperException.Code.SESSIONEXPIRED.intValue())
+                U.warn(log, "Failed to execute async operation, connection lost [path=" + path + ']');
+            else
+                op.cb.processResult(rc, path, ctx, stat);
+        }
+    }
+
+    /**
+     *
+     */
+    private class ConnectionTimeoutTask extends TimerTask {
+        /** */
+        private final long connectStartTime;
+
+        /**
+         * @param connectStartTime Time was connection started.
+         */
+        ConnectionTimeoutTask(long connectStartTime) {
+            this.connectStartTime = connectStartTime;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            boolean connLoss = false;
+
+            synchronized (stateMux) {
+                if (closing)
+                    return;
+
+                if (state == ConnectionState.Disconnected &&
+                    ZookeeperClient.this.connStartTime == connectStartTime) {
+
+                    state = ConnectionState.Lost;
+
+                    U.warn(log, "Failed to establish ZooKeeper connection, close client " +
+                        "[timeout=" + connLossTimeout + ']');
+
+                    connLoss = true;
+                }
+            }
+
+            if (connLoss) {
+                closeClient();
+
+                notifyConnectionLost();
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    private enum ConnectionState {
+        /** */
+        Connected,
+        /** */
+        Disconnected,
+        /** */
+        Lost
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientFailedException.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientFailedException.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientFailedException.java
new file mode 100644
index 0000000..01d011b
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientFailedException.java
@@ -0,0 +1,40 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+/**
+ *
+ */
+class ZookeeperClientFailedException extends Exception {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * @param msg Message.
+     */
+    ZookeeperClientFailedException(String msg) {
+        super(msg);
+    }
+
+    /**
+     * @param cause Cause.
+     */
+    ZookeeperClientFailedException(Throwable cause) {
+        super(cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java
new file mode 100644
index 0000000..3cb5fad
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java
@@ -0,0 +1,362 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.cache.CacheMetrics;
+import org.apache.ignite.cluster.ClusterMetrics;
+import org.apache.ignite.internal.IgniteNodeAttributes;
+import org.apache.ignite.internal.managers.discovery.IgniteClusterNode;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.lang.IgniteProductVersion;
+import org.apache.ignite.spi.discovery.DiscoveryMetricsProvider;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_DAEMON;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_NODE_CONSISTENT_ID;
+
+/**
+ * Zookeeper Cluster Node.
+ */
+public class ZookeeperClusterNode implements IgniteClusterNode, Serializable, Comparable<ZookeeperClusterNode> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private static final byte CLIENT_NODE_MASK = 0x01;
+
+    /** */
+    private UUID id;
+
+    /** */
+    private Serializable consistentId;
+
+    /** */
+    private long internalId;
+
+    /** */
+    private long order;
+
+    /** */
+    private IgniteProductVersion ver;
+
+    /** Node attributes. */
+    private Map<String, Object> attrs;
+
+    /** Internal discovery addresses as strings. */
+    private Collection<String> addrs;
+
+    /** Internal discovery host names as strings. */
+    private Collection<String> hostNames;
+
+    /** */
+    private long sesTimeout;
+
+    /** Metrics provider. */
+    private transient DiscoveryMetricsProvider metricsProvider;
+
+    /** */
+    private transient boolean loc;
+
+    /** */
+    private transient volatile ClusterMetrics metrics;
+
+    /** Node cache metrics. */
+    @GridToStringExclude
+    private transient volatile Map<Integer, CacheMetrics> cacheMetrics;
+
+    /** */
+    private byte flags;
+
+    /** Daemon node flag. */
+    @GridToStringExclude
+    private transient boolean daemon;
+
+    /** Daemon node initialization flag. */
+    @GridToStringExclude
+    private transient volatile boolean daemonInit;
+
+    /**
+     * @param id Node ID.
+     * @param addrs Node addresses.
+     * @param hostNames Node host names.
+     * @param ver Node version.
+     * @param attrs Node attributes.
+     * @param consistentId Consistent ID.
+     * @param sesTimeout Zookeeper session timeout.
+     * @param client Client node flag.
+     * @param metricsProvider Metrics provider.
+     */
+    public ZookeeperClusterNode(
+        UUID id,
+        Collection<String> addrs,
+        Collection<String> hostNames,
+        IgniteProductVersion ver,
+        Map<String, Object> attrs,
+        Serializable consistentId,
+        long sesTimeout,
+        boolean client,
+        DiscoveryMetricsProvider metricsProvider
+    ) {
+        assert id != null;
+        assert consistentId != null;
+
+        this.id = id;
+        this.ver = ver;
+        this.attrs = Collections.unmodifiableMap(attrs);
+        this.addrs = addrs;
+        this.hostNames = hostNames;
+        this.consistentId = consistentId;
+        this.sesTimeout = sesTimeout;
+        this.metricsProvider = metricsProvider;
+
+        if (client)
+            flags |= CLIENT_NODE_MASK;
+    }
+
+    /** {@inheritDoc} */
+    @Override public UUID id() {
+        return id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object consistentId() {
+        return consistentId;
+    }
+
+    /** {@inheritDoc} */
+    public void setConsistentId(Serializable consistentId) {
+        this.consistentId = consistentId;
+
+        final Map<String, Object> map = new HashMap<>(attrs);
+
+        map.put(ATTR_NODE_CONSISTENT_ID, consistentId);
+
+        attrs = Collections.unmodifiableMap(map);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isCacheClient() {
+        return isClient();
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T> T attribute(String name) {
+        // Even though discovery SPI removes this attribute after authentication, keep this check for safety.
+        if (IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS.equals(name))
+            return null;
+
+        return (T)attrs.get(name);
+    }
+
+    /**
+     * Sets node attributes.
+     *
+     * @param attrs Node attributes.
+     */
+    void setAttributes(Map<String, Object> attrs) {
+        this.attrs = U.sealMap(attrs);
+    }
+
+    /**
+     * Gets node attributes without filtering.
+     *
+     * @return Node attributes without filtering.
+     */
+    Map<String, Object> getAttributes() {
+        return attrs;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterMetrics metrics() {
+        if (metricsProvider != null) {
+            ClusterMetrics metrics0 = metricsProvider.metrics();
+
+            assert metrics0 != null;
+
+            metrics = metrics0;
+
+            return metrics0;
+        }
+
+        return metrics;
+    }
+
+    /** {@inheritDoc} */
+    public void setMetrics(ClusterMetrics metrics) {
+        assert metrics != null;
+
+        this.metrics = metrics;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<Integer, CacheMetrics> cacheMetrics() {
+        if (metricsProvider != null) {
+            Map<Integer, CacheMetrics> cacheMetrics0 = metricsProvider.cacheMetrics();
+
+            cacheMetrics = cacheMetrics0;
+
+            return cacheMetrics0;
+        }
+
+        return cacheMetrics;
+    }
+
+    /** {@inheritDoc} */
+    public void setCacheMetrics(Map<Integer, CacheMetrics> cacheMetrics) {
+        this.cacheMetrics = cacheMetrics != null ? cacheMetrics : Collections.<Integer, CacheMetrics>emptyMap();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<String, Object> attributes() {
+        // Even though discovery SPI removes this attribute after authentication, keep this check for safety.
+        return F.view(attrs, new IgnitePredicate<String>() {
+            @Override public boolean apply(String s) {
+                return !IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS.equals(s);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<String> addresses() {
+        return addrs;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<String> hostNames() {
+        return hostNames;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long order() {
+        return order;
+    }
+
+    /**
+     * @return Internal ID corresponds to Zookeeper sequential node.
+     */
+    long internalId() {
+        return internalId;
+    }
+
+    /**
+     * @param internalId Internal ID corresponds to Zookeeper sequential node.
+     */
+    void internalId(long internalId) {
+        this.internalId = internalId;
+    }
+
+    /**
+     * @param order Node order.
+     */
+    void order(long order) {
+        assert order > 0 : order;
+
+        this.order = order;
+    }
+
+    /**
+     * @param newId New node ID.
+     */
+    public void onClientDisconnected(UUID newId) {
+        id = newId;
+    }
+
+    /**
+     * @return Session timeout.
+     */
+    long sessionTimeout() {
+        return sesTimeout;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteProductVersion version() {
+        return ver;
+    }
+
+    /**
+     * @param loc Local node flag.
+     */
+    public void local(boolean loc) {
+        this.loc = loc;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isLocal() {
+        return loc;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isDaemon() {
+        if (!daemonInit) {
+            daemon = "true".equalsIgnoreCase((String)attribute(ATTR_DAEMON));
+
+            daemonInit = true;
+        }
+
+        return daemon;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isClient() {
+        return (CLIENT_NODE_MASK & flags) != 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compareTo(@Nullable ZookeeperClusterNode node) {
+        if (node == null)
+            return 1;
+
+        int res = Long.compare(order, node.order);
+
+        if (res == 0) {
+            assert id().equals(node.id()) : "Duplicate order [this=" + this + ", other=" + node + ']';
+
+            res = id().compareTo(node.id());
+        }
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return id.hashCode();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object obj) {
+        return F.eqNodes(this, obj);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "ZookeeperClusterNode [id=" + id +
+            ", addrs=" + addrs +
+            ", order=" + order +
+            ", loc=" + loc +
+            ", client=" + isClient() + ']';
+    }
+}


[08/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverAbstractTest.java
index 3e98051..37292ff 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverAbstractTest.java
@@ -29,6 +29,7 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteClientDisconnectedException;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.Event;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
@@ -87,9 +88,9 @@ public abstract class IgniteClientReconnectFailoverAbstractTest extends IgniteCl
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
-        TestTcpDiscoverySpi srvSpi = spi(srv);
+        IgniteDiscoverySpi srvSpi = spi0(srv);
 
         final AtomicBoolean stop = new AtomicBoolean(false);
 
@@ -209,14 +210,17 @@ public abstract class IgniteClientReconnectFailoverAbstractTest extends IgniteCl
             }
 
             if (err != null) {
-                log.error(err);
+                log.error("Test error: " + err);
 
                 U.dumpThreads(log);
 
                 CyclicBarrier barrier0 = barrier;
 
-                if (barrier0 != null)
+                if (barrier0 != null) {
+                    barrier = null;
+
                     barrier0.reset();
+                }
 
                 stop.set(true);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectServicesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectServicesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectServicesTest.java
index 3e961e5..1e6dd64 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectServicesTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectServicesTest.java
@@ -65,7 +65,7 @@ public class IgniteClientReconnectServicesTest extends IgniteClientReconnectAbst
 
         assertEquals((Object)topVer, srvc.test());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         reconnectClientNode(client, srv, null);
 
@@ -88,7 +88,7 @@ public class IgniteClientReconnectServicesTest extends IgniteClientReconnectAbst
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         IgniteServices clnServices = client.services();
 
@@ -132,7 +132,7 @@ public class IgniteClientReconnectServicesTest extends IgniteClientReconnectAbst
 
         final IgniteServices services = client.services();
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         BlockTcpCommunicationSpi commSpi = commSpi(srv);
 
@@ -179,7 +179,7 @@ public class IgniteClientReconnectServicesTest extends IgniteClientReconnectAbst
 
         final IgniteServices services = client.services();
 
-        final Ignite srv = clientRouter(client);
+        final Ignite srv = ignite(0);
 
         services.deployClusterSingleton("testReconnectInProgress", new TestServiceImpl());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStopTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStopTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStopTest.java
index e863cdf..b5c3ee8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStopTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStopTest.java
@@ -23,8 +23,10 @@ import org.apache.ignite.IgniteClientDisconnectedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.events.Event;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
 
 import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED;
 import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_RECONNECTED;
@@ -50,15 +52,19 @@ public class IgniteClientReconnectStopTest extends IgniteClientReconnectAbstract
 
         Ignite srv = clientRouter(client);
 
-        TestTcpDiscoverySpi srvSpi = spi(srv);
+        DiscoverySpi srvSpi = spi0(srv);
         final CountDownLatch disconnectLatch = new CountDownLatch(1);
         final CountDownLatch reconnectLatch = new CountDownLatch(1);
 
-        final TestTcpDiscoverySpi clientSpi = spi(client);
+        final IgniteDiscoverySpi clientSpi = spi0(client);
+
+        DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
+
+        clientSpi.setInternalListener(lsnr);
 
         log.info("Block reconnect.");
 
-        clientSpi.writeLatch = new CountDownLatch(1);
+        lsnr.startBlockJoin();
 
         client.events().localListen(new IgnitePredicate<Event>() {
             @Override public boolean apply(Event evt) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStreamerTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStreamerTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStreamerTest.java
index 3959feb..36b9890 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStreamerTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStreamerTest.java
@@ -71,7 +71,7 @@ public class IgniteClientReconnectStreamerTest extends IgniteClientReconnectAbst
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         final IgniteCache<Object, Object> srvCache = srv.cache(CACHE_NAME);
 
@@ -135,7 +135,7 @@ public class IgniteClientReconnectStreamerTest extends IgniteClientReconnectAbst
 
         assertTrue(client.cluster().localNode().isClient());
 
-        Ignite srv = clientRouter(client);
+        Ignite srv = ignite(0);
 
         final IgniteCache<Object, Object> srvCache = srv.cache(CACHE_NAME);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java
index a5d42e9..8edbb52 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java
@@ -256,6 +256,9 @@ public class IgniteClientRejoinTest extends GridCommonAbstractTest {
 
         Ignite srv1 = startGrid("server1");
 
+        if (!tcpDiscovery())
+            return;
+
         crd = ((IgniteKernal)srv1).localNode();
 
         Ignite srv2 = startGrid("server2");

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
index a8afa8b..8fad640 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
@@ -178,16 +178,20 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
      * Waits while topology on all nodes became equals to the expected size.
      *
      * @param nodesCnt Expected nodes count.
-     * @throws InterruptedException If interrupted.
+     * @throws Exception If interrupted.
      */
     @SuppressWarnings("BusyWait")
-    private void awaitDiscovery(long nodesCnt) throws InterruptedException {
-        for (Ignite g : alive) {
-            ((TcpDiscoverySpi)g.configuration().getDiscoverySpi()).waitForClientMessagePrecessed();
+    private void awaitDiscovery(int nodesCnt) throws Exception {
+        if (tcpDiscovery()) {
+            for (Ignite g : alive) {
+                ((TcpDiscoverySpi)g.configuration().getDiscoverySpi()).waitForClientMessagePrecessed();
 
-            while (g.cluster().nodes().size() != nodesCnt)
-                Thread.sleep(10);
+                while (g.cluster().nodes().size() != nodesCnt)
+                    Thread.sleep(10);
+            }
         }
+        else
+            waitForTopology(nodesCnt);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java
index 1d70246..aa2abae 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java
@@ -91,12 +91,12 @@ public abstract class GridAffinityProcessorAbstractSelfTest extends GridCommonAb
     @Override protected void beforeTestsStarted() throws Exception {
         assert NODES_CNT >= 1;
 
-        withCache = false;
+        withCache = true;
 
         for (int i = 0; i < NODES_CNT; i++)
             startGrid(i);
 
-        withCache = true;
+        withCache = false;
 
         for (int i = NODES_CNT; i < 2 * NODES_CNT; i++)
             startGrid(i);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java
index aefbc23..b8f9d70 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java
@@ -27,9 +27,9 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.managers.discovery.IgniteClusterNode;
 import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgnitePredicate;
-import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.events.EventType.EVT_NODE_METRICS_UPDATED;
@@ -103,7 +103,7 @@ public class CacheMetricsForClusterGroupSelfTest extends GridCommonAbstractTest
             Collection<ClusterNode> nodes = grid(0).cluster().forRemotes().nodes();
 
             for (ClusterNode node : nodes) {
-                Map<Integer, CacheMetrics> metrics = ((TcpDiscoveryNode)node).cacheMetrics();
+                Map<Integer, CacheMetrics> metrics = ((IgniteClusterNode)node).cacheMetrics();
                 assertNotNull(metrics);
                 assertFalse(metrics.isEmpty());
             }
@@ -118,6 +118,8 @@ public class CacheMetricsForClusterGroupSelfTest extends GridCommonAbstractTest
 
     /**
      * Test cluster group metrics in case of statistics disabled.
+     *
+     * @throws Exception If failed.
      */
     public void testMetricsStatisticsDisabled() throws Exception {
         createCaches(false);
@@ -134,7 +136,7 @@ public class CacheMetricsForClusterGroupSelfTest extends GridCommonAbstractTest
             Collection<ClusterNode> nodes = grid(0).cluster().forRemotes().nodes();
 
             for (ClusterNode node : nodes) {
-                Map<Integer, CacheMetrics> metrics = ((TcpDiscoveryNode) node).cacheMetrics();
+                Map<Integer, CacheMetrics> metrics = ((IgniteClusterNode)node).cacheMetrics();
                 assertNotNull(metrics);
                 assertTrue(metrics.isEmpty());
             }
@@ -172,7 +174,9 @@ public class CacheMetricsForClusterGroupSelfTest extends GridCommonAbstractTest
     }
 
     /**
-     * Wait for {@link EventType#EVT_NODE_METRICS_UPDATED} event will be receieved.
+     * Wait for {@link EventType#EVT_NODE_METRICS_UPDATED} event will be received.
+     *
+     * @throws InterruptedException If interrupted.
      */
     private void awaitMetricsUpdate() throws InterruptedException {
         final CountDownLatch latch = new CountDownLatch((GRID_CNT + 1) * 2);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
index df93ae4..299dbf4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
@@ -115,6 +115,8 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
     protected void initStoreStrategy() throws IgniteCheckedException {
         if (storeStgy == null)
             storeStgy = isMultiJvm() ? new H2CacheStoreStrategy() : new MapCacheStoreStrategy();
+        else if (isMultiJvm() && !(storeStgy instanceof H2CacheStoreStrategy))
+            storeStgy = new H2CacheStoreStrategy();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
index 0069110..c135f2d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
@@ -45,7 +45,9 @@ public class IgniteCacheNearLockValueSelfTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
-        startGridsMultiThreaded(2);
+        startGrid(1);
+
+        startGrid(0);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
index f321453..55ff31a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
@@ -67,6 +67,17 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
     }
 
     /** {@inheritDoc} */
+    @Override protected void startGrids() throws Exception {
+        int cnt = gridCount();
+
+        assert cnt >= 1 : "At least one grid must be started";
+
+        startGridsMultiThreaded(1, cnt - 1);
+
+        startGrid(0);
+    }
+
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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
index 2337329..838e56d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
@@ -38,6 +38,7 @@ import org.apache.ignite.internal.IgniteClientReconnectAbstractTest;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage;
@@ -1109,6 +1110,70 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
     }
 
     /**
+     * @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 {
+        testReconnectSpi = 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++)
+            ((IgniteDiscoverySpi)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) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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 566860d..2f9bd53 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
@@ -79,7 +79,7 @@ public class IgniteDaemonNodeMarshallerCacheTest extends GridCommonAbstractTest
      * @param startFirst If {@code true} daemon node is started first.
      * @throws Exception If failed.
      */
-    public void marshalOnDaemonNode(boolean startFirst) throws Exception {
+    private void marshalOnDaemonNode(boolean startFirst) throws Exception {
         int nodeIdx = 0;
 
         if (!startFirst) {
@@ -92,6 +92,7 @@ public class IgniteDaemonNodeMarshallerCacheTest extends GridCommonAbstractTest
 
         Ignite daemonNode = startGrid(nodeIdx++);
 
+        assertTrue(daemonNode.cluster().localNode().isDaemon());
         assertEquals("true", daemonNode.cluster().localNode().attribute(ATTR_DAEMON));
 
         daemon = false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataUpdatesFlowTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataUpdatesFlowTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataUpdatesFlowTest.java
index 3ee51c8..7e8c086 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataUpdatesFlowTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataUpdatesFlowTest.java
@@ -184,6 +184,7 @@ public class BinaryMetadataUpdatesFlowTest extends GridCommonAbstractTest {
      * Starts new ignite node and submits computation job to it.
      * @param idx Index.
      * @param stopFlag Stop flag.
+     * @throws Exception If failed.
      */
     private void startComputation(int idx, AtomicBoolean stopFlag) throws Exception {
         clientMode = false;
@@ -199,6 +200,7 @@ public class BinaryMetadataUpdatesFlowTest extends GridCommonAbstractTest {
      * @param idx Index.
      * @param deafClient Deaf client.
      * @param observedIds Observed ids.
+     * @throws Exception If failed.
      */
     private void startListening(int idx, boolean deafClient, Set<Integer> observedIds) throws Exception {
         clientMode = true;
@@ -269,7 +271,7 @@ public class BinaryMetadataUpdatesFlowTest extends GridCommonAbstractTest {
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testFlowNoConflicts() throws Exception {
         startComputation(0, stopFlag0);
@@ -311,11 +313,14 @@ public class BinaryMetadataUpdatesFlowTest extends GridCommonAbstractTest {
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testFlowNoConflictsWithClients() throws Exception {
         startComputation(0, stopFlag0);
 
+        if (!tcpDiscovery())
+            return;
+
         startComputation(1, stopFlag1);
 
         startComputation(2, stopFlag2);
@@ -617,6 +622,9 @@ public class BinaryMetadataUpdatesFlowTest extends GridCommonAbstractTest {
             while (!updatesQueue.isEmpty()) {
                 BinaryUpdateDescription desc = updatesQueue.poll();
 
+                if (desc == null)
+                    break;
+
                 BinaryObjectBuilder builder = ignite.binary().builder(BINARY_TYPE_NAME);
 
                 BinaryObject bo = newBinaryObject(builder, desc);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
index 313aaf9..81614cb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
@@ -242,7 +242,7 @@ public class GridCacheClientNodeBinaryObjectMetadataMultinodeTest extends GridCo
                 @Override public boolean apply() {
                     Collection<BinaryType> metaCol = p0.types();
 
-                    return metaCol.size() == 1000;
+                    return metaCol.size() >= 1000;
                 }
             }, getTestTimeout());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueClientDisconnectTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueClientDisconnectTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueClientDisconnectTest.java
index ed54377..dac3ff6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueClientDisconnectTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueClientDisconnectTest.java
@@ -32,6 +32,9 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+/**
+ *
+ */
 public class GridCacheQueueClientDisconnectTest extends GridCommonAbstractTest {
     /** */
     private static final String IGNITE_QUEUE_NAME = "ignite-queue-client-reconnect-test";
@@ -66,6 +69,10 @@ public class GridCacheQueueClientDisconnectTest extends GridCommonAbstractTest {
         return cfg;
     }
 
+    /**
+     * @param cacheAtomicityMode Atomicity mode.
+     * @return Configuration.
+     */
     private static CollectionConfiguration collectionConfiguration(CacheAtomicityMode cacheAtomicityMode) {
         CollectionConfiguration colCfg = new CollectionConfiguration();
 
@@ -74,6 +81,9 @@ public class GridCacheQueueClientDisconnectTest extends GridCommonAbstractTest {
         return colCfg;
     }
 
+    /**
+     * @throws Exception If failed.
+     */
     public void testClientDisconnect() throws Exception {
         try {
             Ignite server = startGrid(0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
index 51764b5..d85201a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
@@ -494,7 +494,8 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
 
         assertTrue(ignite.configuration().isClientMode());
 
-        assertEquals(clientDiscovery(), ignite.configuration().getDiscoverySpi().isClientMode());
+        if (tcpDiscovery())
+            assertEquals(clientDiscovery(), ignite.configuration().getDiscoverySpi().isClientMode());
 
         return ignite;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java
index 0704dbd..e456047 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java
@@ -52,6 +52,7 @@ import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.internal.DiscoverySpiTestListener;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.cluster.NodeOrderComparator;
 import org.apache.ignite.internal.cluster.NodeOrderLegacyComparator;
@@ -60,7 +61,7 @@ import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.IgniteNodeAttributes;
 import org.apache.ignite.internal.TestRecordingCommunicationSpi;
 import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException;
-import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.affinity.GridAffinityFunctionContextImpl;
 import org.apache.ignite.internal.processors.cache.CacheAffinityChangeMessage;
@@ -88,7 +89,6 @@ import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.services.Service;
 import org.apache.ignite.services.ServiceContext;
-import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
 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;
@@ -158,7 +158,7 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
 
         cfg.setCommunicationSpi(commSpi);
 
-        TestTcpDiscoverySpi discoSpi = new TestTcpDiscoverySpi();
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
 
         discoSpi.setForceServerMode(forceSrvMode);
         discoSpi.setIpFinder(ipFinder);
@@ -674,9 +674,11 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
 
             checkAffinity(4, topVer(4, 0), true);
 
-            TestTcpDiscoverySpi discoSpi = (TestTcpDiscoverySpi)ignite0.configuration().getDiscoverySpi();
+            DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
 
-            discoSpi.blockCustomEvent();
+            ((IgniteDiscoverySpi)ignite0.configuration().getDiscoverySpi()).setInternalListener(lsnr);
+
+            lsnr.blockCustomEvent(CacheAffinityChangeMessage.class);
 
             stopGrid(1);
 
@@ -687,7 +689,7 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
             for (IgniteInternalFuture<?> fut : futs)
                 assertFalse(fut.isDone());
 
-            discoSpi.stopBlock();
+            lsnr.stopBlockCustomEvents();
 
             checkAffinity(3, topVer(5, 0), false);
 
@@ -1409,8 +1411,10 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
     public void testDelayAssignmentAffinityChanged() throws Exception {
         Ignite ignite0 = startServer(0, 1);
 
-        TestTcpDiscoverySpi discoSpi0 =
-            (TestTcpDiscoverySpi)ignite0.configuration().getDiscoverySpi();
+        DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
+
+        ((IgniteDiscoverySpi)ignite0.configuration().getDiscoverySpi()).setInternalListener(lsnr);
+
         TestRecordingCommunicationSpi commSpi0 =
             (TestRecordingCommunicationSpi)ignite0.configuration().getCommunicationSpi();
 
@@ -1418,19 +1422,19 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
 
         checkAffinity(2, topVer(2, 0), true);
 
-        discoSpi0.blockCustomEvent();
+        lsnr.blockCustomEvent(CacheAffinityChangeMessage.class);
 
         startServer(2, 3);
 
         checkAffinity(3, topVer(3, 0), false);
 
-        discoSpi0.waitCustomEvent();
+        lsnr.waitCustomEvent();
 
         blockSupplySend(commSpi0, CACHE_NAME1);
 
         startServer(3, 4);
 
-        discoSpi0.stopBlock();
+        lsnr.stopBlockCustomEvents();
 
         checkAffinity(4, topVer(4, 0), false);
 
@@ -1452,8 +1456,10 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
         try {
             Ignite ignite0 = startServer(0, 1);
 
-            TestTcpDiscoverySpi discoSpi0 =
-                (TestTcpDiscoverySpi)ignite0.configuration().getDiscoverySpi();
+            DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
+
+            ((IgniteDiscoverySpi)ignite0.configuration().getDiscoverySpi()).setInternalListener(lsnr);
+
             TestRecordingCommunicationSpi commSpi0 =
                 (TestRecordingCommunicationSpi)ignite0.configuration().getCommunicationSpi();
 
@@ -1465,11 +1471,11 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
 
             checkAffinity(3, topVer(3, 1), false);
 
-            discoSpi0.blockCustomEvent();
+            lsnr.blockCustomEvent(CacheAffinityChangeMessage.class);
 
             stopNode(2, 4);
 
-            discoSpi0.waitCustomEvent();
+            lsnr.waitCustomEvent();
 
             blockSupplySend(commSpi0, CACHE_NAME1);
 
@@ -1483,7 +1489,7 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
 
             Thread.sleep(2_000);
 
-            discoSpi0.stopBlock();
+            lsnr.stopBlockCustomEvents();
 
             boolean started = GridTestUtils.waitForCondition(new GridAbsPredicate() {
                 @Override public boolean apply() {
@@ -1534,14 +1540,16 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
 
         ignite0.createCache(ccfg);
 
-        TestTcpDiscoverySpi discoSpi0 =
-            (TestTcpDiscoverySpi)ignite0.configuration().getDiscoverySpi();
+        DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
+
+        ((IgniteDiscoverySpi)ignite0.configuration().getDiscoverySpi()).setInternalListener(lsnr);
+
         TestRecordingCommunicationSpi spi =
             (TestRecordingCommunicationSpi)ignite0.configuration().getCommunicationSpi();
 
         blockSupplySend(spi, CACHE_NAME2);
 
-        discoSpi0.blockCustomEvent();
+        lsnr.blockCustomEvent(CacheAffinityChangeMessage.class);
 
         startServer(1, 2);
 
@@ -1551,7 +1559,7 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
 
         spi.stopBlock();
 
-        discoSpi0.waitCustomEvent();
+        lsnr.waitCustomEvent();
 
         ignite0.destroyCache(CACHE_NAME2);
 
@@ -1561,7 +1569,7 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
 
         ignite0.createCache(ccfg);
 
-        discoSpi0.stopBlock();
+        lsnr.stopBlockCustomEvents();
 
         checkAffinity(3, topVer(3, 1), false);
         checkAffinity(3, topVer(3, 2), false);
@@ -2967,83 +2975,6 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
     /**
      *
      */
-    static class TestTcpDiscoverySpi extends TcpDiscoverySpi {
-        /** */
-        private boolean blockCustomEvt;
-
-        /** */
-        private final Object mux = new Object();
-
-        /** */
-        private List<DiscoverySpiCustomMessage> blockedMsgs = new ArrayList<>();
-
-        /** {@inheritDoc} */
-        @Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException {
-            synchronized (mux) {
-                if (blockCustomEvt) {
-                    DiscoveryCustomMessage msg0 = GridTestUtils.getFieldValue(msg, "delegate");
-
-                    if (msg0 instanceof CacheAffinityChangeMessage) {
-                        log.info("Block custom message: " + msg0);
-
-                        blockedMsgs.add(msg);
-
-                        mux.notifyAll();
-
-                        return;
-                    }
-                }
-            }
-
-            super.sendCustomEvent(msg);
-        }
-
-        /**
-         *
-         */
-        public void blockCustomEvent() {
-            synchronized (mux) {
-                assert blockedMsgs.isEmpty() : blockedMsgs;
-
-                blockCustomEvt = true;
-            }
-        }
-
-        /**
-         * @throws InterruptedException If interrupted.
-         */
-        public void waitCustomEvent() throws InterruptedException {
-            synchronized (mux) {
-                while (blockedMsgs.isEmpty())
-                    mux.wait();
-            }
-        }
-
-        /**
-         *
-         */
-        public void stopBlock() {
-            List<DiscoverySpiCustomMessage> msgs;
-
-            synchronized (this) {
-                msgs = new ArrayList<>(blockedMsgs);
-
-                blockCustomEvt = false;
-
-                blockedMsgs.clear();
-            }
-
-            for (DiscoverySpiCustomMessage msg : msgs) {
-                log.info("Resend blocked message: " + msg);
-
-                super.sendCustomEvent(msg);
-            }
-        }
-    }
-
-    /**
-     *
-     */
     static class TestEntryProcessor implements EntryProcessor<Object, Object, Object> {
         /** */
         private Object val;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java
index 3834df9..5dea5d9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java
@@ -45,6 +45,7 @@ import org.apache.ignite.transactions.TransactionIsolation;
 import static org.apache.ignite.IgniteState.STOPPED;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_TX_SALVAGE_TIMEOUT;
 import static org.apache.ignite.IgniteSystemProperties.getInteger;
+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.transactions.TransactionConcurrency.OPTIMISTIC;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
@@ -188,7 +189,7 @@ public abstract class GridCacheNodeFailureAbstractTest extends GridCommonAbstrac
 
                     return true;
                 }
-            }, EVT_NODE_LEFT);
+            }, EVT_NODE_LEFT, EVT_NODE_FAILED);
 
             stopGrid(idx);
 
@@ -268,7 +269,7 @@ public abstract class GridCacheNodeFailureAbstractTest extends GridCommonAbstrac
 
                 return true;
             }
-        }, EVT_NODE_LEFT);
+        }, EVT_NODE_LEFT, EVT_NODE_FAILED);
 
         stopGrid(idx);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java
index e71d3ee..b7ae844 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java
@@ -168,6 +168,8 @@ public class IgniteCache150ClientsTest extends GridCommonAbstractTest {
 
         log.info("Started all clients.");
 
+        waitForTopology(CLIENTS + 1);
+
         checkNodes(CLIENTS + 1);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java
index a0be40e..7785a3c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java
@@ -178,7 +178,7 @@ public class IgniteCacheManyClientsTest extends GridCommonAbstractTest {
         log.info("All clients started.");
 
         try {
-            checkNodes(SRVS + CLIENTS);
+            checkNodes0(SRVS + CLIENTS);
         }
         finally {
             for (Ignite client : clients)
@@ -188,6 +188,30 @@ public class IgniteCacheManyClientsTest extends GridCommonAbstractTest {
 
     /**
      * @param expCnt Expected number of nodes.
+     * @throws Exception If failed.
+     */
+    private void checkNodes0(final int expCnt) throws Exception {
+        boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                try {
+                    checkNodes(expCnt);
+
+                    return true;
+                }
+                catch (AssertionFailedError e) {
+                    log.info("Check failed, will retry: " + e);
+                }
+
+                return false;
+            }
+        }, 10_000);
+
+        if (!wait)
+            checkNodes(expCnt);
+    }
+
+    /**
+     * @param expCnt Expected number of nodes.
      */
     private void checkNodes(int expCnt) {
         assertEquals(expCnt, G.allGrids().size());
@@ -297,23 +321,7 @@ public class IgniteCacheManyClientsTest extends GridCommonAbstractTest {
             if (err0 != null)
                 throw err0;
 
-            boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() {
-                @Override public boolean apply() {
-                    try {
-                        checkNodes(SRVS + THREADS);
-
-                        return true;
-                    }
-                    catch (AssertionFailedError e) {
-                        log.info("Check failed, will retry: " + e);
-                    }
-
-                    return false;
-                }
-            }, 10_000);
-
-            if (!wait)
-                checkNodes(SRVS + THREADS);
+            checkNodes0(SRVS + THREADS);
 
             log.info("Stop clients.");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteOptimisticTxSuspendResumeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteOptimisticTxSuspendResumeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteOptimisticTxSuspendResumeTest.java
index a55f21d..486fd60 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteOptimisticTxSuspendResumeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteOptimisticTxSuspendResumeTest.java
@@ -713,6 +713,8 @@ public class IgniteOptimisticTxSuspendResumeTest extends GridCommonAbstractTest
                 ", backups=" + ccfg.getBackups() +
                 ", near=" + (ccfg.getNearConfiguration() != null) + "]");
 
+            awaitPartitionMapExchange();
+
             int srvNum = serversNumber();
             if (serversNumber() > 1) {
                 ignite(serversNumber() + 1).createNearCache(ccfg.getName(), new NearCacheConfiguration<>());

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
index e3fa116..60f2f0a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
@@ -116,6 +116,8 @@ public class GridCacheDhtPreloadMultiThreadedSelfTest extends GridCommonAbstract
                     @Nullable @Override public Object call() throws Exception {
                         IgniteConfiguration cfg = loadConfiguration("modules/core/src/test/config/spring-multicache.xml");
 
+                        cfg.setGridLogger(getTestResources().getLogger());
+
                         startGrid(Thread.currentThread().getName(), cfg);
 
                         return null;
@@ -161,6 +163,8 @@ public class GridCacheDhtPreloadMultiThreadedSelfTest extends GridCommonAbstract
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = loadConfiguration("modules/core/src/test/config/spring-multicache.xml");
 
+        cfg.setGridLogger(getTestResources().getLogger());
+
         cfg.setIgniteInstanceName(igniteInstanceName);
 
         for (CacheConfiguration cCfg : cfg.getCacheConfiguration()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
index 05a9759..83eff89 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
@@ -308,6 +308,8 @@ public class GridCacheDhtPreloadSelfTest extends GridCommonAbstractTest {
 
                 // Check all left nodes.
                 checkActiveState(ignites);
+
+                awaitPartitionMapExchange(); // Need wait, otherwise test logic is broken if EVT_NODE_FAILED exchanges are merged.
             }
 
             info("Waiting for preload futures: " + F.view(futs, new IgnitePredicate<IgniteFuture<?>>() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java
index 7b350c8..060af21 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java
@@ -34,11 +34,11 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.spi.IgniteSpiException;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -65,7 +65,6 @@ public class TxRecoveryStoreEnabledTest extends GridCommonAbstractTest {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
         cfg.setCommunicationSpi(new TestCommunicationSpi());
-        cfg.setDiscoverySpi(new TestDiscoverySpi());
 
         CacheConfiguration ccfg = defaultCacheConfiguration();
 
@@ -126,7 +125,7 @@ public class TxRecoveryStoreEnabledTest extends GridCommonAbstractTest {
                     IgniteConfiguration cfg = node0.configuration();
 
                     ((TestCommunicationSpi)cfg.getCommunicationSpi()).block();
-                    ((TestDiscoverySpi)cfg.getDiscoverySpi()).simulateNodeFailure();
+                    ((IgniteDiscoverySpi)cfg.getDiscoverySpi()).simulateNodeFailure();
                 }
                 catch (InterruptedException e) {
                     Thread.currentThread().interrupt();
@@ -201,16 +200,6 @@ public class TxRecoveryStoreEnabledTest extends GridCommonAbstractTest {
     /**
      *
      */
-    private static class TestDiscoverySpi extends TcpDiscoverySpi {
-        /** {@inheritDoc} */
-        @Override protected void simulateNodeFailure() {
-            super.simulateNodeFailure();
-        }
-    }
-
-    /**
-     *
-     */
     private static class TestCommunicationSpi extends TcpCommunicationSpi {
         /** Block. */
         private volatile boolean block;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
index 3c57957..96fb8f6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
@@ -37,6 +37,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 
 /**
@@ -125,7 +126,7 @@ public class GridCachePartitionedExplicitLockNodeFailureSelfTest extends GridCom
 
                         return true;
                     }
-                }, EVT_NODE_LEFT));
+                }, EVT_NODE_LEFT, EVT_NODE_FAILED));
         }
 
         stopGrid(idx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ClientReconnectContinuousQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ClientReconnectContinuousQueryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ClientReconnectContinuousQueryTest.java
index c8b3bb6..9b531c6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ClientReconnectContinuousQueryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ClientReconnectContinuousQueryTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.query.continuous;
 
+import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import javax.cache.event.CacheEntryListenerException;
@@ -28,6 +29,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.managers.communication.GridIoManager;
 import org.apache.ignite.internal.util.nio.GridNioServer;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -90,7 +92,7 @@ public class ClientReconnectContinuousQueryTest extends GridCommonAbstractTest {
         try {
             startGrids(2);
 
-            IgniteEx client = grid(CLIENT_IDX);
+            final IgniteEx client = grid(CLIENT_IDX);
 
             client.events().localListen(new DisconnectListener(), EventType.EVT_CLIENT_NODE_DISCONNECTED);
 
@@ -112,11 +114,19 @@ public class ClientReconnectContinuousQueryTest extends GridCommonAbstractTest {
 
             skipRead(client, true);
 
-            putSomeKeys(1_000);
+            IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    assertTrue(disconLatch.await(10_000, TimeUnit.MILLISECONDS));
+
+                    skipRead(client, false);
 
-            assertTrue(disconLatch.await(10_000, TimeUnit.MILLISECONDS));
+                    return null;
+                }
+            });
 
-            skipRead(client, false);
+            putSomeKeys(1_000);
+
+            fut.get();
 
             assertTrue(reconLatch.await(10_000, TimeUnit.MILLISECONDS));
 
@@ -129,7 +139,6 @@ public class ClientReconnectContinuousQueryTest extends GridCommonAbstractTest {
         finally {
             stopAllGrids();
         }
-
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryRemoteFilterMissingInClassPathSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryRemoteFilterMissingInClassPathSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryRemoteFilterMissingInClassPathSelfTest.java
index 92c1760..226302f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryRemoteFilterMissingInClassPathSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryRemoteFilterMissingInClassPathSelfTest.java
@@ -104,7 +104,7 @@ public class ContinuousQueryRemoteFilterMissingInClassPathSelfTest extends GridC
         setExternalLoader = true;
         final Ignite ignite0 = startGrid(1);
 
-        executeContiniouseQuery(ignite0.cache("simple"));
+        executeContinuousQuery(ignite0.cache("simple"));
 
         log = new GridStringLogger();
         clientMode = true;
@@ -112,8 +112,10 @@ public class ContinuousQueryRemoteFilterMissingInClassPathSelfTest extends GridC
 
         startGrid(2);
 
-        assertTrue(log.toString().contains("Failed to unmarshal continuous query remote filter on client node. " +
-            "Can be ignored."));
+        String logStr = log.toString();
+
+        assertTrue(logStr.contains("Failed to unmarshal continuous query remote filter on client node. " +
+            "Can be ignored.") || logStr.contains("Failed to unmarshal continuous routine handler"));
     }
 
     /**
@@ -127,7 +129,7 @@ public class ContinuousQueryRemoteFilterMissingInClassPathSelfTest extends GridC
         clientMode = false;
         final Ignite ignite0 = startGrid(1);
 
-        executeContiniouseQuery(ignite0.cache("simple"));
+        executeContinuousQuery(ignite0.cache("simple"));
 
         log = new GridStringLogger();
         clientMode = true;
@@ -149,15 +151,18 @@ public class ContinuousQueryRemoteFilterMissingInClassPathSelfTest extends GridC
         setExternalLoader = true;
         final Ignite ignite0 = startGrid(1);
 
-        executeContiniouseQuery(ignite0.cache("simple"));
+        executeContinuousQuery(ignite0.cache("simple"));
 
         log = new GridStringLogger();
         setExternalLoader = false;
 
         startGrid(2);
 
-        assertTrue(log.toString().contains("class org.apache.ignite.IgniteCheckedException: " +
-            "Failed to find class with given class loader for unmarshalling"));
+        String logStr = log.toString();
+
+        assertTrue(logStr.contains("class org.apache.ignite.IgniteCheckedException: " +
+            "Failed to find class with given class loader for unmarshalling")
+            || logStr.contains("Failed to unmarshal continuous routine handler"));
     }
 
     /**
@@ -171,7 +176,7 @@ public class ContinuousQueryRemoteFilterMissingInClassPathSelfTest extends GridC
         setExternalLoader = true;
         final Ignite ignite0 = startGrid(1);
 
-        executeContiniouseQuery(ignite0.cache("simple"));
+        executeContinuousQuery(ignite0.cache("simple"));
 
         log = new GridStringLogger();
 
@@ -185,7 +190,7 @@ public class ContinuousQueryRemoteFilterMissingInClassPathSelfTest extends GridC
      * @param cache Ignite cache.
      * @throws Exception If fail.
      */
-    private void executeContiniouseQuery(IgniteCache cache) throws Exception {
+    private void executeContinuousQuery(IgniteCache cache) throws Exception {
         ContinuousQuery<Integer, String> qry = new ContinuousQuery<>();
 
         qry.setLocalListener(

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryClientReconnectTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryClientReconnectTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryClientReconnectTest.java
index 9ad6d4e..906cc7d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryClientReconnectTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryClientReconnectTest.java
@@ -119,6 +119,9 @@ public class IgniteCacheContinuousQueryClientReconnectTest extends IgniteClientR
      * @throws Exception If failed.
      */
     public void testReconnectClientAndLeftRouter() throws Exception {
+        if (!tcpDiscovery())
+            return;
+
         Ignite client = grid(serverCount());
 
         final Ignite srv = clientRouter(client);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java
index 61ceef7..16ea848 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java
@@ -18,11 +18,12 @@
 package org.apache.ignite.internal.processors.cache.version;
 
 import java.util.HashSet;
+import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
 import javax.cache.Cache;
 import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.EntryProcessorResult;
 import javax.cache.processor.MutableEntry;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheEntry;
@@ -56,23 +57,15 @@ public abstract class CacheVersionedEntryAbstractTest extends GridCacheAbstractS
     public void testInvoke() throws Exception {
         Cache<Integer, String> cache = grid(0).cache(DEFAULT_CACHE_NAME);
 
-        final AtomicInteger invoked = new AtomicInteger();
-
-        cache.invoke(100, new EntryProcessor<Integer, String, Object>() {
-            @Override public Object process(MutableEntry<Integer, String> entry, Object... arguments)
-                throws EntryProcessorException {
-
-                invoked.incrementAndGet();
-
+        assertNotNull(cache.invoke(100, new EntryProcessor<Integer, String, Object>() {
+            @Override public Object process(MutableEntry<Integer, String> entry, Object... args) {
                 CacheEntry<Integer, String> verEntry = entry.unwrap(CacheEntry.class);
 
                 checkVersionedEntry(verEntry);
 
-                return entry;
+                return verEntry.version();
             }
-        });
-
-        assert invoked.get() > 0;
+        }));
     }
 
     /**
@@ -86,23 +79,17 @@ public abstract class CacheVersionedEntryAbstractTest extends GridCacheAbstractS
         for (int i = 0; i < ENTRIES_NUM; i++)
             keys.add(i);
 
-        final AtomicInteger invoked = new AtomicInteger();
-
-        cache.invokeAll(keys, new EntryProcessor<Integer, String, Object>() {
-            @Override public Object process(MutableEntry<Integer, String> entry, Object... arguments)
-                throws EntryProcessorException {
-
-                invoked.incrementAndGet();
-
+        Map<Integer, EntryProcessorResult<Object>> res = cache.invokeAll(keys, new EntryProcessor<Integer, String, Object>() {
+            @Override public Object process(MutableEntry<Integer, String> entry, Object... args) {
                 CacheEntry<Integer, String> verEntry = entry.unwrap(CacheEntry.class);
 
                 checkVersionedEntry(verEntry);
 
-                return null;
+                return verEntry.version();
             }
         });
 
-        assert invoked.get() > 0;
+        assertEquals(ENTRIES_NUM, res.size());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
index f07b1a3..1a7abd4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
@@ -447,7 +447,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
 
             grid(0).compute().broadcast(F.noop());
 
-            assert latch.await(2, SECONDS);
+            assert latch.await(10, SECONDS) : latch;
 
             assertEquals(GRID_CNT, nodeIds.size());
             assertEquals(GRID_CNT, cnt.get());
@@ -488,7 +488,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
 
             grid(0).compute().broadcast(F.noop());
 
-            assert latch.await(2, SECONDS);
+            assert latch.await(10, SECONDS) : latch;
 
             assertEquals(GRID_CNT, nodeIds.size());
             assertEquals(GRID_CNT, cnt.get());
@@ -532,7 +532,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
 
             grid(0).compute().broadcast(F.noop());
 
-            assert latch.await(2, SECONDS);
+            assert latch.await(10, SECONDS) : latch;
 
             assertEquals(GRID_CNT, nodeIds.size());
             assertEquals(GRID_CNT, cnt.get());
@@ -578,7 +578,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
             grid(0).compute().broadcast(F.noop());
             grid(0).compute().withName("exclude").run(F.noop());
 
-            assert latch.await(2, SECONDS);
+            assert latch.await(10, SECONDS) : latch;
 
             assertEquals(GRID_CNT, nodeIds.size());
             assertEquals(GRID_CNT, cnt.get());
@@ -619,7 +619,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
 
             grid(0).compute().broadcast(F.noop());
 
-            assert latch.await(2, SECONDS);
+            assert latch.await(10, SECONDS) : latch;
 
             assertEquals(GRID_CNT - 1, nodeIds.size());
             assertEquals(GRID_CNT - 1, cnt.get());
@@ -660,7 +660,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
 
             grid(0).compute().broadcast(F.noop());
 
-            assert latch.await(2, SECONDS);
+            assert latch.await(10, SECONDS) : latch;
 
             assertEquals(GRID_CNT - 1, nodeIds.size());
             assertEquals(GRID_CNT - 1, cnt.get());
@@ -701,7 +701,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
 
             grid(0).compute().broadcast(F.noop());
 
-            assert latch.await(2, SECONDS);
+            assert latch.await(10, SECONDS) : latch;
 
             assertEquals(1, nodeIds.size());
             assertEquals(1, cnt.get());
@@ -744,7 +744,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
 
             grid(0).compute().broadcast(F.noop());
 
-            assert latch.await(2, SECONDS);
+            assert latch.await(10, SECONDS) : latch;
 
             assertEquals(1, nodeIds.size());
             assertEquals(1, cnt.get());
@@ -785,7 +785,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
 
             grid(0).compute().run(F.noop());
 
-            assert latch.await(2, SECONDS);
+            assert latch.await(10, SECONDS) : latch;
 
             assertEquals(1, nodeIds.size());
             assertEquals(1, cnt.get());
@@ -828,7 +828,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
 
         compute(grid(0).cluster().forLocal()).run(F.noop());
 
-        assert latch.await(2, SECONDS);
+        assert latch.await(10, SECONDS) : latch;
 
         assertEquals(1, cnt.get());
 
@@ -878,7 +878,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
 
             grid(0).compute().broadcast(F.noop());
 
-            assert latch.await(2, SECONDS);
+            assert latch.await(10, SECONDS) : latch;
 
             assertEquals(GRID_CNT + 1, nodeIds.size());
             assertEquals(GRID_CNT + 1, cnt.get());
@@ -929,7 +929,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
 
             grid(0).compute().broadcast(F.noop());
 
-            assert latch.await(2, SECONDS);
+            assert latch.await(10, SECONDS) : latch;
 
             assertEquals(GRID_CNT, nodeIds.size());
             assertEquals(GRID_CNT, cnt.get());
@@ -980,7 +980,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
 
             grid(0).compute().broadcast(F.noop());
 
-            assert latch.await(2, SECONDS);
+            assert latch.await(10, SECONDS) : latch;
 
             assertEquals(GRID_CNT + 1, nodeIds.size());
             assertEquals(GRID_CNT + 1, cnt.get());
@@ -1036,7 +1036,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
 
             grid(0).compute().broadcast(F.noop());
 
-            assert latch.await(2, SECONDS);
+            assert latch.await(10, SECONDS) : latch;
 
             assertEquals(GRID_CNT, nodeIds.size());
             assertEquals(GRID_CNT, cnt.get());
@@ -1145,7 +1145,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void testMultithreadedWithNodeRestart() throws Exception {
+    public void _testMultithreadedWithNodeRestart() throws Exception {
         final AtomicBoolean stop = new AtomicBoolean();
         final BlockingQueue<IgniteBiTuple<Integer, UUID>> queue = new LinkedBlockingQueue<>();
         final Collection<UUID> started = new GridConcurrentHashSet<>();
@@ -1153,9 +1153,11 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
 
         final Random rnd = new Random();
 
+        final int consumeCnt = tcpDiscovery() ? CONSUME_CNT : CONSUME_CNT / 2;
+
         IgniteInternalFuture<?> starterFut = multithreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
-                for (int i = 0; i < CONSUME_CNT; i++) {
+                for (int i = 0; i < consumeCnt; i++) {
                     int idx = rnd.nextInt(GRID_CNT);
 
                     try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
index c1af323..8f03c4c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
@@ -49,6 +49,9 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
     /** Number of grids started for tests. */
     private static final int NODES_CNT = 4;
 
+    /** */
+    private static final int CLIENT_IDX = 1;
+
     /** Test singleton service name. */
     private static final String SINGLETON_NAME = "testSingleton";
 
@@ -61,11 +64,11 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
 
         cfg.setMarshaller(new BinaryMarshaller());
 
-        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(ipFinder).setForceServerMode(true));
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
 
         cfg.setCacheConfiguration();
 
-        if (igniteInstanceName.equals(getTestIgniteInstanceName(0)))
+        if (igniteInstanceName.equals(getTestIgniteInstanceName(CLIENT_IDX)))
             cfg.setClientMode(true);
 
         return cfg;
@@ -88,8 +91,10 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
     public void testDefaultClosure() throws Exception {
         Set<String> srvNames = new HashSet<>(NODES_CNT - 1);
 
-        for (int i = 1; i < NODES_CNT; ++i)
-            srvNames.add(getTestIgniteInstanceName(i));
+        for (int i = 0; i < NODES_CNT; ++i) {
+            if (i != CLIENT_IDX)
+                srvNames.add(getTestIgniteInstanceName(i));
+        }
 
         for (int i = 0 ; i < NODES_CNT; i++) {
             log.info("Iteration: " + i);
@@ -137,7 +142,7 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
 
             assertEquals(1, res.size());
 
-            assertEquals(getTestIgniteInstanceName(0), F.first(res));
+            assertEquals(getTestIgniteInstanceName(CLIENT_IDX), F.first(res));
         }
     }
 
@@ -168,7 +173,7 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDefaultService() throws Exception {
-        UUID clientNodeId = grid(0).cluster().localNode().id();
+        UUID clientNodeId = grid(CLIENT_IDX).cluster().localNode().id();
 
         for (int i = 0 ; i < NODES_CNT; i++) {
             log.info("Iteration: " + i);
@@ -209,7 +214,7 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testClientService() throws Exception {
-        UUID clientNodeId = grid(0).cluster().localNode().id();
+        UUID clientNodeId = grid(CLIENT_IDX).cluster().localNode().id();
 
         for (int i = 0 ; i < NODES_CNT; i++) {
             log.info("Iteration: " + i);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/util/GridTestClockTimer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/GridTestClockTimer.java b/modules/core/src/test/java/org/apache/ignite/internal/util/GridTestClockTimer.java
index 5da9042..2d26b72 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/GridTestClockTimer.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/GridTestClockTimer.java
@@ -32,6 +32,15 @@ public class GridTestClockTimer implements Runnable {
         }
     }
 
+    /**
+     * @return {@code True} if need start test time.
+     */
+    public static boolean startTestTimer() {
+        synchronized (IgniteUtils.mux) {
+            return IgniteUtils.gridCnt == 0;
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public void run() {
         while (true) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerMappingConsistencyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerMappingConsistencyTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerMappingConsistencyTest.java
index cd8e757..78f3c03 100644
--- a/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerMappingConsistencyTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerMappingConsistencyTest.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 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;
 
 /**
@@ -157,6 +158,9 @@ public class GridMarshallerMappingConsistencyTest extends GridCommonAbstractTest
         Ignite g2 = startGrid(2);
         startGrid(1);
 
+        assertTrue("Failed to wait for automatic grid activation",
+            GridTestUtils.waitForCondition(() -> g2.cluster().active(), getTestTimeout()));
+
         IgniteCache<Integer, DummyObject> c2 = g2.cache(CACHE_NAME);
 
         assertEquals(k, c2.get(k).val);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java
index 7541cec..a7c4521 100644
--- a/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java
@@ -24,7 +24,6 @@ import java.io.ObjectOutput;
 import java.io.Serializable;
 import java.net.URL;
 import java.net.URLClassLoader;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -37,22 +36,20 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteMessaging;
 import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.DiscoverySpiTestListener;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
 import org.apache.ignite.internal.processors.continuous.StartRoutineDiscoveryMessage;
+import org.apache.ignite.internal.processors.continuous.StartRoutineDiscoveryMessageV2;
 import org.apache.ignite.internal.processors.continuous.StopRoutineDiscoveryMessage;
-import org.apache.ignite.internal.processors.marshaller.MappingAcceptedMessage;
-import org.apache.ignite.internal.processors.marshaller.MappingProposedMessage;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.typedef.P2;
 import org.apache.ignite.internal.util.typedef.PA;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.resources.IgniteInstanceResource;
-import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
 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;
@@ -206,11 +203,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
-        TestTcpDiscoverySpi discoSpi = new TestTcpDiscoverySpi();
-
-        discoSpi.setIpFinder(ipFinder);
-
-        cfg.setDiscoverySpi(discoSpi);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
 
         return cfg;
     }
@@ -1036,7 +1029,11 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
     public void testAsyncOld() throws Exception {
         final AtomicInteger msgCnt = new AtomicInteger();
 
-        TestTcpDiscoverySpi discoSpi = (TestTcpDiscoverySpi)ignite2.configuration().getDiscoverySpi();
+        IgniteDiscoverySpi discoSpi = (IgniteDiscoverySpi)ignite2.configuration().getDiscoverySpi();
+
+        DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
+
+        discoSpi.setInternalListener(lsnr);
 
         assertFalse(ignite2.message().isAsync());
 
@@ -1054,7 +1051,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
             }
         }, IllegalStateException.class, null);
 
-        discoSpi.blockCustomEvent();
+        lsnr.blockCustomEvent(StartRoutineDiscoveryMessage.class, StartRoutineDiscoveryMessageV2.class);
 
         final String topic = "topic";
 
@@ -1079,7 +1076,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
 
         Assert.assertFalse(starFut.isDone());
 
-        discoSpi.stopBlock();
+        lsnr.stopBlockCustomEvents();
 
         GridTestUtils.assertThrows(log, new Callable<Void>() {
             @Override public Void call() throws Exception {
@@ -1095,7 +1092,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
 
         Assert.assertTrue(starFut.isDone());
 
-        discoSpi.blockCustomEvent();
+        lsnr.blockCustomEvent(StopRoutineDiscoveryMessage.class);
 
         message(ignite1.cluster().forRemotes()).send(topic, "msg1");
 
@@ -1125,7 +1122,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
 
         Assert.assertFalse(stopFut.isDone());
 
-        discoSpi.stopBlock();
+        lsnr.stopBlockCustomEvents();
 
         stopFut.get();
 
@@ -1144,9 +1141,13 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
     public void testAsync() throws Exception {
         final AtomicInteger msgCnt = new AtomicInteger();
 
-        TestTcpDiscoverySpi discoSpi = (TestTcpDiscoverySpi)ignite2.configuration().getDiscoverySpi();
+        IgniteDiscoverySpi discoSpi = (IgniteDiscoverySpi)ignite2.configuration().getDiscoverySpi();
 
-        discoSpi.blockCustomEvent();
+        DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
+
+        discoSpi.setInternalListener(lsnr);
+
+        lsnr.blockCustomEvent(StartRoutineDiscoveryMessage.class, StartRoutineDiscoveryMessageV2.class);
 
         final String topic = "topic";
 
@@ -1167,7 +1168,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
 
         Assert.assertFalse(starFut.isDone());
 
-        discoSpi.stopBlock();
+        lsnr.stopBlockCustomEvents();
 
         UUID id = starFut.get();
 
@@ -1175,7 +1176,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
 
         Assert.assertTrue(starFut.isDone());
 
-        discoSpi.blockCustomEvent();
+        lsnr.blockCustomEvent(StopRoutineDiscoveryMessage.class);
 
         message(ignite1.cluster().forRemotes()).send(topic, "msg1");
 
@@ -1195,7 +1196,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
 
         Assert.assertFalse(stopFut.isDone());
 
-        discoSpi.stopBlock();
+        lsnr.stopBlockCustomEvents();
 
         stopFut.get();
 
@@ -1209,89 +1210,6 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
     }
 
     /**
-     *
-     */
-    static class TestTcpDiscoverySpi extends TcpDiscoverySpi {
-        /** */
-        private boolean blockCustomEvt;
-
-        /** */
-        private final Object mux = new Object();
-
-        /** */
-        private List<DiscoverySpiCustomMessage> blockedMsgs = new ArrayList<>();
-
-        /** {@inheritDoc} */
-        @Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException {
-            synchronized (mux) {
-                if (blockCustomEvt) {
-                    DiscoveryCustomMessage msg0 = GridTestUtils.getFieldValue(msg, "delegate");
-
-                    if (msg0 instanceof MappingProposedMessage || msg0 instanceof MappingAcceptedMessage){
-                        super.sendCustomEvent(msg);
-
-                        return;
-                    }
-
-                    if (msg0 instanceof StopRoutineDiscoveryMessage || msg0 instanceof StartRoutineDiscoveryMessage) {
-                        log.info("Block custom message: " + msg0);
-
-                        blockedMsgs.add(msg);
-
-                        mux.notifyAll();
-
-                        return;
-                    }
-                }
-            }
-
-            super.sendCustomEvent(msg);
-        }
-
-        /**
-         *
-         */
-        public void blockCustomEvent() {
-            synchronized (mux) {
-                assert blockedMsgs.isEmpty() : blockedMsgs;
-
-                blockCustomEvt = true;
-            }
-        }
-
-        /**
-         * @throws InterruptedException If interrupted.
-         */
-        public void waitCustomEvent() throws InterruptedException {
-            synchronized (mux) {
-                while (blockedMsgs.isEmpty())
-                    mux.wait();
-            }
-        }
-
-        /**
-         *
-         */
-        public void stopBlock() {
-            List<DiscoverySpiCustomMessage> msgs;
-
-            synchronized (this) {
-                msgs = new ArrayList<>(blockedMsgs);
-
-                blockCustomEvt = false;
-
-                blockedMsgs.clear();
-            }
-
-            for (DiscoverySpiCustomMessage msg : msgs) {
-                log.info("Resend blocked message: " + msg);
-
-                super.sendCustomEvent(msg);
-            }
-        }
-    }
-
-    /**
      * Tests that message listener registers only for one oldest node.
      *
      * @throws Exception If an error occurred.


[02/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiTest.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiTest.java
new file mode 100644
index 0000000..fb12c3a
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiTest.java
@@ -0,0 +1,4847 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.curator.test.TestingCluster;
+import org.apache.curator.test.TestingZooKeeperServer;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.CommunicationFailureContext;
+import org.apache.ignite.configuration.CommunicationFailureResolver;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.WALMode;
+import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.events.Event;
+import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.DiscoverySpiTestListener;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.IgnitionEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.managers.discovery.CustomEventListener;
+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.discovery.GridDiscoveryManager;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpiInternalListener;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.GridCacheAbstractFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.TestCacheNodeExcludingFilter;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage;
+import org.apache.ignite.internal.processors.security.SecurityContext;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.T3;
+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.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.lang.IgniteOutClosure;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.marshaller.jdk.JdkMarshaller;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.security.SecurityCredentials;
+import org.apache.ignite.plugin.security.SecurityPermission;
+import org.apache.ignite.plugin.security.SecuritySubject;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.apache.ignite.resources.LoggerResource;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator;
+import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi;
+import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpiTestSuite2;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZKUtil;
+import org.apache.zookeeper.ZkTestClientCnxnSocketNIO;
+import org.apache.zookeeper.ZooKeeper;
+import org.jetbrains.annotations.Nullable;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED;
+import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_RECONNECTED;
+import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
+import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
+import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGNITE_INSTANCE_NAME;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2;
+import static org.apache.ignite.spi.discovery.zk.internal.ZookeeperDiscoveryImpl.IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD;
+import static org.apache.zookeeper.ZooKeeper.ZOOKEEPER_CLIENT_CNXN_SOCKET;
+
+/**
+ *
+ */
+@SuppressWarnings("deprecation")
+public class ZookeeperDiscoverySpiTest extends GridCommonAbstractTest {
+    /** */
+    private static final String IGNITE_ZK_ROOT = ZookeeperDiscoverySpi.DFLT_ROOT_PATH;
+
+    /** */
+    private static final int ZK_SRVS = 3;
+
+    /** */
+    private static TestingCluster zkCluster;
+
+    /** To run test with real local ZK. */
+    private static final boolean USE_TEST_CLUSTER = true;
+
+    /** */
+    private boolean client;
+
+    /** */
+    private static ThreadLocal<Boolean> clientThreadLoc = new ThreadLocal<>();
+
+    /** */
+    private static ConcurrentHashMap<UUID, Map<Long, DiscoveryEvent>> evts = new ConcurrentHashMap<>();
+
+    /** */
+    private static volatile boolean err;
+
+    /** */
+    private boolean testSockNio;
+
+    /** */
+    private boolean testCommSpi;
+
+    /** */
+    private long sesTimeout;
+
+    /** */
+    private long joinTimeout;
+
+    /** */
+    private boolean clientReconnectDisabled;
+
+    /** */
+    private ConcurrentHashMap<String, ZookeeperDiscoverySpi> spis = new ConcurrentHashMap<>();
+
+    /** */
+    private Map<String, Object> userAttrs;
+
+    /** */
+    private boolean dfltConsistenId;
+
+    /** */
+    private UUID nodeId;
+
+    /** */
+    private boolean persistence;
+
+    /** */
+    private IgniteOutClosure<CommunicationFailureResolver> commFailureRslvr;
+
+    /** */
+    private IgniteOutClosure<DiscoverySpiNodeAuthenticator> auth;
+
+    /** */
+    private String zkRootPath;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(final String igniteInstanceName) throws Exception {
+        if (testSockNio)
+            System.setProperty(ZOOKEEPER_CLIENT_CNXN_SOCKET, ZkTestClientCnxnSocketNIO.class.getName());
+
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        if (nodeId != null)
+            cfg.setNodeId(nodeId);
+
+        if (!dfltConsistenId)
+            cfg.setConsistentId(igniteInstanceName);
+
+        ZookeeperDiscoverySpi zkSpi = new ZookeeperDiscoverySpi();
+
+        if (joinTimeout != 0)
+            zkSpi.setJoinTimeout(joinTimeout);
+
+        zkSpi.setSessionTimeout(sesTimeout > 0 ? sesTimeout : 10_000);
+
+        zkSpi.setClientReconnectDisabled(clientReconnectDisabled);
+
+        // Set authenticator for basic sanity tests.
+        if (auth != null) {
+            zkSpi.setAuthenticator(auth.apply());
+
+            zkSpi.setInternalListener(new IgniteDiscoverySpiInternalListener() {
+                @Override public void beforeJoin(ClusterNode locNode, IgniteLogger log) {
+                    ZookeeperClusterNode locNode0 = (ZookeeperClusterNode)locNode;
+
+                    Map<String, Object> attrs = new HashMap<>(locNode0.getAttributes());
+
+                    attrs.put(ATTR_SECURITY_CREDENTIALS, new SecurityCredentials(null, null, igniteInstanceName));
+
+                    locNode0.setAttributes(attrs);
+                }
+
+                @Override public boolean beforeSendCustomEvent(DiscoverySpi spi, IgniteLogger log, DiscoverySpiCustomMessage msg) {
+                    return false;
+                }
+            });
+        }
+
+        spis.put(igniteInstanceName, zkSpi);
+
+        if (USE_TEST_CLUSTER) {
+            assert zkCluster != null;
+
+            zkSpi.setZkConnectionString(zkCluster.getConnectString());
+
+            if (zkRootPath != null)
+                zkSpi.setZkRootPath(zkRootPath);
+        }
+        else
+            zkSpi.setZkConnectionString("localhost:2181");
+
+        cfg.setDiscoverySpi(zkSpi);
+
+        CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME);
+
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        Boolean clientMode = clientThreadLoc.get();
+
+        if (clientMode != null)
+            cfg.setClientMode(clientMode);
+        else
+            cfg.setClientMode(client);
+
+        if (userAttrs != null)
+            cfg.setUserAttributes(userAttrs);
+
+        Map<IgnitePredicate<? extends Event>, int[]> lsnrs = new HashMap<>();
+
+        lsnrs.put(new IgnitePredicate<Event>() {
+            /** */
+            @IgniteInstanceResource
+            private Ignite ignite;
+
+            @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+            @Override public boolean apply(Event evt) {
+                try {
+                    DiscoveryEvent discoveryEvt = (DiscoveryEvent)evt;
+
+                    UUID locId = ((IgniteKernal)ignite).context().localNodeId();
+
+                    Map<Long, DiscoveryEvent> nodeEvts = evts.get(locId);
+
+                    if (nodeEvts == null) {
+                        Object old = evts.put(locId, nodeEvts = new TreeMap<>());
+
+                        assertNull(old);
+
+                        synchronized (nodeEvts) {
+                            DiscoveryLocalJoinData locJoin = ((IgniteKernal)ignite).context().discovery().localJoin();
+
+                            nodeEvts.put(locJoin.event().topologyVersion(), locJoin.event());
+                        }
+                    }
+
+                    synchronized (nodeEvts) {
+                        DiscoveryEvent old = nodeEvts.put(discoveryEvt.topologyVersion(), discoveryEvt);
+
+                        assertNull(old);
+                    }
+                }
+                catch (Throwable e) {
+                    error("Unexpected error [evt=" + evt + ", err=" + e + ']', e);
+
+                    err = true;
+                }
+
+                return true;
+            }
+        }, new int[]{EVT_NODE_JOINED, EVT_NODE_FAILED, EVT_NODE_LEFT});
+
+        cfg.setLocalEventListeners(lsnrs);
+
+        if (persistence) {
+            DataStorageConfiguration memCfg = new DataStorageConfiguration()
+                .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setMaxSize(100 * 1024 * 1024).
+                    setPersistenceEnabled(true))
+                .setPageSize(1024)
+                .setWalMode(WALMode.LOG_ONLY);
+
+            cfg.setDataStorageConfiguration(memCfg);
+        }
+
+        if (testCommSpi)
+            cfg.setCommunicationSpi(new ZkTestCommunicationSpi());
+
+        if (commFailureRslvr != null)
+            cfg.setCommunicationFailureResolver(commFailureRslvr.apply());
+
+        return cfg;
+    }
+
+    /**
+     * @param clientMode Client mode flag for started nodes.
+     */
+    private void clientMode(boolean clientMode) {
+        client = clientMode;
+    }
+
+    /**
+     * @param clientMode Client mode flag for nodes started from current thread.
+     */
+    private void clientModeThreadLocal(boolean clientMode) {
+        clientThreadLoc.set(clientMode);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        System.setProperty(ZookeeperDiscoveryImpl.IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_TIMEOUT, "1000");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopZkCluster();
+
+        System.clearProperty(ZookeeperDiscoveryImpl.IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_TIMEOUT);
+
+        super.afterTestsStopped();
+    }
+
+    /**
+     *
+     */
+    private void stopZkCluster() {
+        if (zkCluster != null) {
+            try {
+                zkCluster.close();
+            }
+            catch (Exception e) {
+                U.error(log, "Failed to stop Zookeeper client: " + e, e);
+            }
+
+            zkCluster = null;
+        }
+    }
+
+    /**
+     *
+     */
+    private static void ackEveryEventSystemProperty() {
+        System.setProperty(IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD, "1");
+    }
+
+    /**
+     *
+     */
+    private void clearAckEveryEventSystemProperty() {
+        System.setProperty(IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD, "1");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        if (USE_TEST_CLUSTER && zkCluster == null) {
+            zkCluster = ZookeeperDiscoverySpiTestSuite2.createTestingCluster(ZK_SRVS);
+
+            zkCluster.start();
+        }
+
+        reset();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        clearAckEveryEventSystemProperty();
+
+        try {
+            assertFalse("Unexpected error, see log for details", err);
+
+            checkEventsConsistency();
+
+            checkInternalStructuresCleanup();
+
+            //TODO uncomment when https://issues.apache.org/jira/browse/IGNITE-8193 is fixed
+//            checkZkNodesCleanup();
+        }
+        finally {
+            reset();
+
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkInternalStructuresCleanup() throws Exception {
+        for (Ignite node : G.allGrids()) {
+            final AtomicReference<?> res = GridTestUtils.getFieldValue(spi(node), "impl", "commErrProcFut");
+
+            GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                @Override public boolean apply() {
+                    return res.get() == null;
+                }
+            }, 30_000);
+
+            assertNull(res.get());
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testZkRootNotExists() throws Exception {
+        zkRootPath = "/a/b/c";
+
+        for (int i = 0; i < 3; i++) {
+            reset();
+
+            startGridsMultiThreaded(5);
+
+            waitForTopology(5);
+
+            stopAllGrids();
+
+            checkEventsConsistency();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMetadataUpdate() throws Exception {
+        startGrid(0);
+
+        GridTestUtils.runMultiThreaded(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                ignite(0).configuration().getMarshaller().marshal(new C1());
+                ignite(0).configuration().getMarshaller().marshal(new C2());
+
+                return null;
+            }
+        }, 64, "marshal");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNodeAddresses() throws Exception {
+        startGridsMultiThreaded(3);
+
+        clientMode(true);
+
+        startGridsMultiThreaded(3, 3);
+
+        waitForTopology(6);
+
+        for (Ignite node : G.allGrids()) {
+            ClusterNode locNode0 = node.cluster().localNode();
+
+            assertTrue(locNode0.addresses().size() > 0);
+            assertTrue(locNode0.hostNames().size() > 0);
+
+            for (ClusterNode node0 : node.cluster().nodes()) {
+                assertTrue(node0.addresses().size() > 0);
+                assertTrue(node0.hostNames().size() > 0);
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSetConsistentId() throws Exception {
+        startGridsMultiThreaded(3);
+
+        clientMode(true);
+
+        startGridsMultiThreaded(3, 3);
+
+        waitForTopology(6);
+
+        for (Ignite node : G.allGrids()) {
+            ClusterNode locNode0 = node.cluster().localNode();
+
+            assertEquals(locNode0.attribute(ATTR_IGNITE_INSTANCE_NAME),
+                locNode0.consistentId());
+
+            for (ClusterNode node0 : node.cluster().nodes()) {
+                assertEquals(node0.attribute(ATTR_IGNITE_INSTANCE_NAME),
+                    node0.consistentId());
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDefaultConsistentId() throws Exception {
+        dfltConsistenId = true;
+
+        startGridsMultiThreaded(3);
+
+        clientMode(true);
+
+        startGridsMultiThreaded(3, 3);
+
+        waitForTopology(6);
+
+        for (Ignite node : G.allGrids()) {
+            ClusterNode locNode0 = node.cluster().localNode();
+
+            assertNotNull(locNode0.consistentId());
+
+            for (ClusterNode node0 : node.cluster().nodes())
+                assertNotNull(node0.consistentId());
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientNodesStatus() throws Exception {
+        startGrid(0);
+
+        for (Ignite node : G.allGrids()) {
+            assertEquals(0, node.cluster().forClients().nodes().size());
+            assertEquals(1, node.cluster().forServers().nodes().size());
+        }
+
+        clientMode(true);
+
+        startGrid(1);
+
+        for (Ignite node : G.allGrids()) {
+            assertEquals(1, node.cluster().forClients().nodes().size());
+            assertEquals(1, node.cluster().forServers().nodes().size());
+        }
+
+        clientMode(false);
+
+        startGrid(2);
+
+        clientMode(true);
+
+        startGrid(3);
+
+        for (Ignite node : G.allGrids()) {
+            assertEquals(2, node.cluster().forClients().nodes().size());
+            assertEquals(2, node.cluster().forServers().nodes().size());
+        }
+
+        stopGrid(1);
+
+        waitForTopology(3);
+
+        for (Ignite node : G.allGrids()) {
+            assertEquals(1, node.cluster().forClients().nodes().size());
+            assertEquals(2, node.cluster().forServers().nodes().size());
+        }
+
+        stopGrid(2);
+
+        waitForTopology(2);
+
+        for (Ignite node : G.allGrids()) {
+            assertEquals(1, node.cluster().forClients().nodes().size());
+            assertEquals(1, node.cluster().forServers().nodes().size());
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void _testLocalAuthenticationFails() throws Exception {
+        auth = ZkTestNodeAuthenticator.factory(getTestIgniteInstanceName(0));
+
+        Throwable err = GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                startGrid(0);
+
+                return null;
+            }
+        }, IgniteCheckedException.class, null);
+
+        IgniteSpiException spiErr = X.cause(err, IgniteSpiException.class);
+
+        assertNotNull(spiErr);
+        assertTrue(spiErr.getMessage().contains("Authentication failed for local node"));
+
+        startGrid(1);
+        startGrid(2);
+
+        checkTestSecuritySubject(2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAuthentication() throws Exception {
+        auth = ZkTestNodeAuthenticator.factory(getTestIgniteInstanceName(1),
+            getTestIgniteInstanceName(5));
+
+        startGrid(0);
+
+        checkTestSecuritySubject(1);
+
+        {
+            clientMode(false);
+            checkStartFail(1);
+
+            clientMode(true);
+            checkStartFail(1);
+
+            clientMode(false);
+        }
+
+        startGrid(2);
+
+        checkTestSecuritySubject(2);
+
+        stopGrid(2);
+
+        checkTestSecuritySubject(1);
+
+        startGrid(2);
+
+        checkTestSecuritySubject(2);
+
+        stopGrid(0);
+
+        checkTestSecuritySubject(1);
+
+        checkStartFail(1);
+
+        clientMode(false);
+
+        startGrid(3);
+
+        clientMode(true);
+
+        startGrid(4);
+
+        clientMode(false);
+
+        startGrid(0);
+
+        checkTestSecuritySubject(4);
+
+        checkStartFail(1);
+        checkStartFail(5);
+
+        clientMode(true);
+
+        checkStartFail(1);
+        checkStartFail(5);
+    }
+
+    /**
+     * @param nodeIdx Node index.
+     */
+    private void checkStartFail(final int nodeIdx) {
+        Throwable err = GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                startGrid(nodeIdx);
+
+                return null;
+            }
+        }, IgniteCheckedException.class, null);
+
+        IgniteSpiException spiErr = X.cause(err, IgniteSpiException.class);
+
+        assertNotNull(spiErr);
+        assertTrue(spiErr.getMessage().contains("Authentication failed"));
+    }
+
+    /**
+     * @param expNodes Expected nodes number.
+     * @throws Exception If failed.
+     */
+    private void checkTestSecuritySubject(int expNodes) throws Exception {
+        waitForTopology(expNodes);
+
+        List<Ignite> nodes = G.allGrids();
+
+        JdkMarshaller marsh = new JdkMarshaller();
+
+        for (Ignite ignite : nodes) {
+            Collection<ClusterNode> nodes0 = ignite.cluster().nodes();
+
+            assertEquals(nodes.size(), nodes0.size());
+
+            for (ClusterNode node : nodes0) {
+                byte[] secSubj = node.attribute(ATTR_SECURITY_SUBJECT_V2);
+
+                assertNotNull(secSubj);
+
+                ZkTestNodeAuthenticator.TestSecurityContext secCtx = marsh.unmarshal(secSubj, null);
+
+                assertEquals(node.attribute(ATTR_IGNITE_INSTANCE_NAME), secCtx.nodeName);
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStopNode_1() throws Exception {
+        startGrids(5);
+
+        waitForTopology(5);
+
+        stopGrid(3);
+
+        waitForTopology(4);
+
+        startGrid(3);
+
+        waitForTopology(5);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCustomEventsSimple1_SingleNode() throws Exception {
+        ackEveryEventSystemProperty();
+
+        Ignite srv0 = startGrid(0);
+
+        srv0.createCache(new CacheConfiguration<>("c1"));
+
+        waitForEventsAcks(srv0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCustomEventsSimple1_5_Nodes() throws Exception {
+        ackEveryEventSystemProperty();
+
+        Ignite srv0 = startGrids(5);
+
+        srv0.createCache(new CacheConfiguration<>("c1"));
+
+        awaitPartitionMapExchange();
+
+        waitForEventsAcks(srv0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCustomEvents_FastStopProcess_1() throws Exception {
+        customEvents_FastStopProcess(1, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCustomEvents_FastStopProcess_2() throws Exception {
+        customEvents_FastStopProcess(5, 5);
+    }
+
+    /**
+     * @param srvs Servers number.
+     * @param clients Clients number.
+     * @throws Exception If failed.
+     */
+    private void customEvents_FastStopProcess(int srvs, int clients) throws Exception {
+        ackEveryEventSystemProperty();
+
+        Map<UUID, List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>>> rcvdMsgs =
+            new ConcurrentHashMap<>();
+
+        Ignite crd = startGrid(0);
+
+        UUID crdId = crd.cluster().localNode().id();
+
+        if (srvs > 1)
+            startGridsMultiThreaded(1, srvs - 1);
+
+        if (clients > 0) {
+            client = true;
+
+            startGridsMultiThreaded(srvs, clients);
+        }
+
+        awaitPartitionMapExchange();
+
+        List<Ignite> nodes = G.allGrids();
+
+        assertEquals(srvs + clients, nodes.size());
+
+        for (Ignite node : nodes)
+            registerTestEventListeners(node, rcvdMsgs);
+
+        int payload = 0;
+
+        AffinityTopologyVersion topVer = ((IgniteKernal)crd).context().discovery().topologyVersionEx();
+
+        for (Ignite node : nodes) {
+            UUID sndId = node.cluster().localNode().id();
+
+            info("Send from node: " + sndId);
+
+            GridDiscoveryManager discoveryMgr = ((IgniteKernal)node).context().discovery();
+
+            {
+                List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>> expCrdMsgs = new ArrayList<>();
+                List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>> expNodesMsgs = Collections.emptyList();
+
+                TestFastStopProcessCustomMessage msg = new TestFastStopProcessCustomMessage(false, payload++);
+
+                expCrdMsgs.add(new T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>(topVer, sndId, msg));
+
+                discoveryMgr.sendCustomEvent(msg);
+
+                doSleep(200); // Wait some time to check extra messages are not received.
+
+                checkEvents(crd, rcvdMsgs, expCrdMsgs);
+
+                for (Ignite node0 : nodes) {
+                    if (node0 != crd)
+                        checkEvents(node0, rcvdMsgs, expNodesMsgs);
+                }
+
+                rcvdMsgs.clear();
+            }
+            {
+                List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>> expCrdMsgs = new ArrayList<>();
+                List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>> expNodesMsgs = new ArrayList<>();
+
+                TestFastStopProcessCustomMessage msg = new TestFastStopProcessCustomMessage(true, payload++);
+
+                expCrdMsgs.add(new T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>(topVer, sndId, msg));
+
+                discoveryMgr.sendCustomEvent(msg);
+
+                TestFastStopProcessCustomMessageAck ackMsg = new TestFastStopProcessCustomMessageAck(msg.payload);
+
+                expCrdMsgs.add(new T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>(topVer, crdId, ackMsg));
+                expNodesMsgs.add(new T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>(topVer, crdId, ackMsg));
+
+                doSleep(200); // Wait some time to check extra messages are not received.
+
+                checkEvents(crd, rcvdMsgs, expCrdMsgs);
+
+                for (Ignite node0 : nodes) {
+                    if (node0 != crd)
+                        checkEvents(node0, rcvdMsgs, expNodesMsgs);
+                }
+
+                rcvdMsgs.clear();
+            }
+
+            waitForEventsAcks(crd);
+        }
+    }
+
+    /**
+     * @param node Node to check.
+     * @param rcvdMsgs Received messages.
+     * @param expMsgs Expected messages.
+     * @throws Exception If failed.
+     */
+    private void checkEvents(
+        Ignite node,
+        final Map<UUID, List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>>> rcvdMsgs,
+        final List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>> expMsgs) throws Exception {
+        final UUID nodeId = node.cluster().localNode().id();
+
+        assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>> msgs = rcvdMsgs.get(nodeId);
+
+                int size = msgs == null ? 0 : msgs.size();
+
+                return size >= expMsgs.size();
+            }
+        }, 5000));
+
+        List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>> msgs = rcvdMsgs.get(nodeId);
+
+        if (msgs == null)
+            msgs = Collections.emptyList();
+
+        assertEqualsCollections(expMsgs, msgs);
+    }
+
+    /**
+     * @param node Node.
+     * @param rcvdMsgs Map to store received events.
+     */
+    private void registerTestEventListeners(Ignite node,
+        final Map<UUID, List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>>> rcvdMsgs) {
+        GridDiscoveryManager discoveryMgr = ((IgniteKernal)node).context().discovery();
+
+        final UUID nodeId = node.cluster().localNode().id();
+
+        discoveryMgr.setCustomEventListener(TestFastStopProcessCustomMessage.class,
+            new CustomEventListener<TestFastStopProcessCustomMessage>() {
+                @Override public void onCustomEvent(AffinityTopologyVersion topVer, ClusterNode snd, TestFastStopProcessCustomMessage msg) {
+                    List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>> list = rcvdMsgs.get(nodeId);
+
+                    if (list == null)
+                        rcvdMsgs.put(nodeId, list = new ArrayList<>());
+
+                    list.add(new T3<>(topVer, snd.id(), (DiscoveryCustomMessage)msg));
+                }
+            }
+        );
+        discoveryMgr.setCustomEventListener(TestFastStopProcessCustomMessageAck.class,
+            new CustomEventListener<TestFastStopProcessCustomMessageAck>() {
+                @Override public void onCustomEvent(AffinityTopologyVersion topVer, ClusterNode snd, TestFastStopProcessCustomMessageAck msg) {
+                    List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>> list = rcvdMsgs.get(nodeId);
+
+                    if (list == null)
+                        rcvdMsgs.put(nodeId, list = new ArrayList<>());
+
+                    list.add(new T3<>(topVer, snd.id(), (DiscoveryCustomMessage)msg));
+                }
+            }
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSegmentation1() throws Exception {
+        sesTimeout = 2000;
+        testSockNio = true;
+
+        Ignite node0 = startGrid(0);
+
+        final CountDownLatch l = new CountDownLatch(1);
+
+        node0.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                l.countDown();
+
+                return false;
+            }
+        }, EventType.EVT_NODE_SEGMENTED);
+
+        ZkTestClientCnxnSocketNIO c0 = ZkTestClientCnxnSocketNIO.forNode(node0);
+
+        c0.closeSocket(true);
+
+        for (int i = 0; i < 10; i++) {
+            Thread.sleep(1_000);
+
+            if (l.getCount() == 0)
+                break;
+        }
+
+        info("Allow connect");
+
+        c0.allowConnect();
+
+        assertTrue(l.await(10, TimeUnit.SECONDS));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSegmentation2() throws Exception {
+        sesTimeout = 2000;
+
+        Ignite node0 = startGrid(0);
+
+        final CountDownLatch l = new CountDownLatch(1);
+
+        node0.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                l.countDown();
+
+                return false;
+            }
+        }, EventType.EVT_NODE_SEGMENTED);
+
+        try {
+            zkCluster.close();
+
+            assertTrue(l.await(10, TimeUnit.SECONDS));
+        }
+        finally {
+            zkCluster = ZookeeperDiscoverySpiTestSuite2.createTestingCluster(ZK_SRVS);
+
+            zkCluster.start();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSegmentation3() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-8183");
+
+        sesTimeout = 5000;
+
+        Ignite node0 = startGrid(0);
+
+        final CountDownLatch l = new CountDownLatch(1);
+
+        node0.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                l.countDown();
+
+                return false;
+            }
+        }, EventType.EVT_NODE_SEGMENTED);
+
+        List<TestingZooKeeperServer> srvs = zkCluster.getServers();
+
+        assertEquals(3, srvs.size());
+
+        try {
+            srvs.get(0).stop();
+            srvs.get(1).stop();
+
+            assertTrue(l.await(20, TimeUnit.SECONDS));
+        }
+        finally {
+            zkCluster.close();
+
+            zkCluster = ZookeeperDiscoverySpiTestSuite2.createTestingCluster(ZK_SRVS);
+
+            zkCluster.start();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQuorumRestore() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-8180");
+
+        sesTimeout = 15_000;
+
+        startGrids(3);
+
+        waitForTopology(3);
+
+        List<TestingZooKeeperServer> srvs = zkCluster.getServers();
+
+        assertEquals(3, srvs.size());
+
+        try {
+            srvs.get(0).stop();
+            srvs.get(1).stop();
+
+            U.sleep(2000);
+
+            srvs.get(1).restart();
+
+            U.sleep(4000);
+
+            startGrid(4);
+
+            waitForTopology(4);
+        }
+        finally {
+            zkCluster.close();
+
+            zkCluster = ZookeeperDiscoverySpiTestSuite2.createTestingCluster(ZK_SRVS);
+
+            zkCluster.start();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionRestore1() throws Exception {
+        testSockNio = true;
+
+        Ignite node0 = startGrid(0);
+
+        ZkTestClientCnxnSocketNIO c0 = ZkTestClientCnxnSocketNIO.forNode(node0);
+
+        c0.closeSocket(false);
+
+        startGrid(1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionRestore2() throws Exception {
+        testSockNio = true;
+
+        Ignite node0 = startGrid(0);
+
+        ZkTestClientCnxnSocketNIO c0 = ZkTestClientCnxnSocketNIO.forNode(node0);
+
+        c0.closeSocket(false);
+
+        startGridsMultiThreaded(1, 5);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionRestore_NonCoordinator1() throws Exception {
+        connectionRestore_NonCoordinator(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionRestore_NonCoordinator2() throws Exception {
+        connectionRestore_NonCoordinator(true);
+    }
+
+    /**
+     * @param failWhenDisconnected {@code True} if fail node while another node is disconnected.
+     * @throws Exception If failed.
+     */
+    private void connectionRestore_NonCoordinator(boolean failWhenDisconnected) throws Exception {
+        testSockNio = true;
+
+        Ignite node0 = startGrid(0);
+        Ignite node1 = startGrid(1);
+
+        ZkTestClientCnxnSocketNIO c1 = ZkTestClientCnxnSocketNIO.forNode(node1);
+
+        c1.closeSocket(true);
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() {
+                try {
+                    startGrid(2);
+                }
+                catch (Exception e) {
+                    info("Start error: " + e);
+                }
+
+                return null;
+            }
+        }, "start-node");
+
+        checkEvents(node0, joinEvent(3));
+
+        if (failWhenDisconnected) {
+            ZookeeperDiscoverySpi spi = spis.get(getTestIgniteInstanceName(2));
+
+            closeZkClient(spi);
+
+            checkEvents(node0, failEvent(4));
+        }
+
+        c1.allowConnect();
+
+        checkEvents(ignite(1), joinEvent(3));
+
+        if (failWhenDisconnected) {
+            checkEvents(ignite(1), failEvent(4));
+
+            IgnitionEx.stop(getTestIgniteInstanceName(2), true, true);
+        }
+
+        fut.get();
+
+        waitForTopology(failWhenDisconnected ? 2 : 3);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionRestore_Coordinator1() throws Exception {
+        connectionRestore_Coordinator(1, 1, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionRestore_Coordinator1_1() throws Exception {
+        connectionRestore_Coordinator(1, 1, 1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionRestore_Coordinator2() throws Exception {
+        connectionRestore_Coordinator(1, 3, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionRestore_Coordinator3() throws Exception {
+        connectionRestore_Coordinator(3, 3, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionRestore_Coordinator4() throws Exception {
+        connectionRestore_Coordinator(3, 3, 1);
+    }
+
+    /**
+     * @param initNodes Number of initially started nodes.
+     * @param startNodes Number of nodes to start after coordinator loose connection.
+     * @param failCnt Number of nodes to stop after coordinator loose connection.
+     * @throws Exception If failed.
+     */
+    private void connectionRestore_Coordinator(final int initNodes, int startNodes, int failCnt) throws Exception {
+        sesTimeout = 30_000;
+        testSockNio = true;
+
+        Ignite node0 = startGrids(initNodes);
+
+        ZkTestClientCnxnSocketNIO c0 = ZkTestClientCnxnSocketNIO.forNode(node0);
+
+        c0.closeSocket(true);
+
+        final AtomicInteger nodeIdx = new AtomicInteger(initNodes);
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+            @Override public Void call() {
+                try {
+                    startGrid(nodeIdx.getAndIncrement());
+                }
+                catch (Exception e) {
+                    error("Start failed: " + e);
+                }
+
+                return null;
+            }
+        }, startNodes, "start-node");
+
+        int cnt = 0;
+
+        DiscoveryEvent[] expEvts = new DiscoveryEvent[startNodes - failCnt];
+
+        int expEvtCnt = 0;
+
+        sesTimeout = 1000;
+
+        List<ZkTestClientCnxnSocketNIO> blockedC = new ArrayList<>();
+
+        final List<String> failedZkNodes = new ArrayList<>(failCnt);
+
+        for (int i = initNodes; i < initNodes + startNodes; i++) {
+            final ZookeeperDiscoverySpi spi = waitSpi(getTestIgniteInstanceName(i));
+
+            assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                @Override public boolean apply() {
+                    Object spiImpl = GridTestUtils.getFieldValue(spi, "impl");
+
+                    if (spiImpl == null)
+                        return false;
+
+                    long internalOrder = GridTestUtils.getFieldValue(spiImpl, "rtState", "internalOrder");
+
+                    return internalOrder > 0;
+                }
+            }, 10_000));
+
+            if (cnt++ < failCnt) {
+                ZkTestClientCnxnSocketNIO c = ZkTestClientCnxnSocketNIO.forNode(getTestIgniteInstanceName(i));
+
+                c.closeSocket(true);
+
+                blockedC.add(c);
+
+                failedZkNodes.add(aliveZkNodePath(spi));
+            }
+            else {
+                expEvts[expEvtCnt] = joinEvent(initNodes + expEvtCnt + 1);
+
+                expEvtCnt++;
+            }
+        }
+
+        waitNoAliveZkNodes(log, zkCluster.getConnectString(), failedZkNodes, 30_000);
+
+        c0.allowConnect();
+
+        for (ZkTestClientCnxnSocketNIO c : blockedC)
+            c.allowConnect();
+
+        if (expEvts.length > 0) {
+            for (int i = 0; i < initNodes; i++)
+                checkEvents(ignite(i), expEvts);
+        }
+
+        fut.get();
+
+        waitForTopology(initNodes + startNodes - failCnt);
+    }
+
+    /**
+     * @param node Node.
+     * @return Corresponding znode.
+     */
+    private static String aliveZkNodePath(Ignite node) {
+        return aliveZkNodePath(node.configuration().getDiscoverySpi());
+    }
+
+    /**
+     * @param spi SPI.
+     * @return Znode related to given SPI.
+     */
+    private static String aliveZkNodePath(DiscoverySpi spi) {
+        String path = GridTestUtils.getFieldValue(spi, "impl", "rtState", "locNodeZkPath");
+
+        return path.substring(path.lastIndexOf('/') + 1);
+    }
+
+    /**
+     * @param log Logger.
+     * @param connectString Zookeeper connect string.
+     * @param failedZkNodes Znodes which should be removed.
+     * @param timeout Timeout.
+     * @throws Exception If failed.
+     */
+    private static void waitNoAliveZkNodes(final IgniteLogger log,
+        String connectString,
+        final List<String> failedZkNodes,
+        long timeout)
+        throws Exception
+    {
+        final ZookeeperClient zkClient = new ZookeeperClient(log, connectString, 10_000, null);
+
+        try {
+            assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                @Override public boolean apply() {
+                    try {
+                        List<String> c = zkClient.getChildren(IGNITE_ZK_ROOT + "/" + ZkIgnitePaths.ALIVE_NODES_DIR);
+
+                        for (String failedZkNode : failedZkNodes) {
+                            if (c.contains(failedZkNode)) {
+                                log.info("Alive node is not removed [node=" + failedZkNode + ", all=" + c + ']');
+
+                                return false;
+                            }
+                        }
+
+                        return true;
+                    }
+                    catch (Exception e) {
+                        e.printStackTrace();
+
+                        fail();
+
+                        return true;
+                    }
+                }
+            }, timeout));
+        }
+        finally {
+            zkClient.close();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConcurrentStartWithClient() throws Exception {
+        final int NODES = 20;
+
+        for (int i = 0; i < 3; i++) {
+            info("Iteration: " + i);
+
+            final int srvIdx = ThreadLocalRandom.current().nextInt(NODES);
+
+            final AtomicInteger idx = new AtomicInteger();
+
+            GridTestUtils.runMultiThreaded(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    int threadIdx = idx.getAndIncrement();
+
+                    clientModeThreadLocal(threadIdx == srvIdx || ThreadLocalRandom.current().nextBoolean());
+
+                    startGrid(threadIdx);
+
+                    return null;
+                }
+            }, NODES, "start-node");
+
+            waitForTopology(NODES);
+
+            stopAllGrids();
+
+            checkEventsConsistency();
+
+            evts.clear();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConcurrentStart() throws Exception {
+        final int NODES = 20;
+
+        for (int i = 0; i < 3; i++) {
+            info("Iteration: " + i);
+
+            final AtomicInteger idx = new AtomicInteger();
+
+            final CyclicBarrier b = new CyclicBarrier(NODES);
+
+            GridTestUtils.runMultiThreaded(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    b.await();
+
+                    int threadIdx = idx.getAndIncrement();
+
+                    startGrid(threadIdx);
+
+                    return null;
+                }
+            }, NODES, "start-node");
+
+            waitForTopology(NODES);
+
+            stopAllGrids();
+
+            checkEventsConsistency();
+
+            evts.clear();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConcurrentStartStop1() throws Exception {
+       concurrentStartStop(1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConcurrentStartStop2() throws Exception {
+        concurrentStartStop(5);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConcurrentStartStop2_EventsThrottle() throws Exception {
+        System.setProperty(ZookeeperDiscoveryImpl.IGNITE_ZOOKEEPER_DISCOVERY_SPI_MAX_EVTS, "1");
+
+        try {
+            concurrentStartStop(5);
+        }
+        finally {
+            System.clearProperty(ZookeeperDiscoveryImpl.IGNITE_ZOOKEEPER_DISCOVERY_SPI_MAX_EVTS);
+        }
+    }
+
+    /**
+     * @param initNodes Number of initially started nnodes.
+     * @throws Exception If failed.
+     */
+    private void concurrentStartStop(final int initNodes) throws Exception {
+        startGrids(initNodes);
+
+        final int NODES = 5;
+
+        long topVer = initNodes;
+
+        for (int i = 0; i < 10; i++) {
+            info("Iteration: " + i);
+
+            DiscoveryEvent[] expEvts = new DiscoveryEvent[NODES];
+
+            startGridsMultiThreaded(initNodes, NODES);
+
+            for (int j = 0; j < NODES; j++)
+                expEvts[j] = joinEvent(++topVer);
+
+            checkEvents(ignite(0), expEvts);
+
+            checkEventsConsistency();
+
+            final CyclicBarrier b = new CyclicBarrier(NODES);
+
+            GridTestUtils.runMultiThreaded(new IgniteInClosure<Integer>() {
+                @Override public void apply(Integer idx) {
+                    try {
+                        b.await();
+
+                        stopGrid(initNodes + idx);
+                    }
+                    catch (Exception e) {
+                        e.printStackTrace();
+
+                        fail();
+                    }
+                }
+            }, NODES, "stop-node");
+
+            for (int j = 0; j < NODES; j++)
+                expEvts[j] = failEvent(++topVer);
+
+            checkEventsConsistency();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClusterRestart() throws Exception {
+        startGridsMultiThreaded(3, false);
+
+        stopAllGrids();
+
+        evts.clear();
+
+        startGridsMultiThreaded(3, false);
+
+        waitForTopology(3);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionRestore4() throws Exception {
+        testSockNio = true;
+
+        Ignite node0 = startGrid(0);
+
+        ZkTestClientCnxnSocketNIO c0 = ZkTestClientCnxnSocketNIO.forNode(node0);
+
+        c0.closeSocket(false);
+
+        startGrid(1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartStop_1_Node() throws Exception {
+        startGrid(0);
+
+        waitForTopology(1);
+
+        stopGrid(0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRestarts_2_Nodes() throws Exception {
+        startGrid(0);
+
+        for (int i = 0; i < 10; i++) {
+            info("Iteration: " + i);
+
+            startGrid(1);
+
+            waitForTopology(2);
+
+            stopGrid(1);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartStop_2_Nodes_WithCache() throws Exception {
+        startGrids(2);
+
+        for (Ignite node : G.allGrids()) {
+            IgniteCache<Object, Object> cache = node.cache(DEFAULT_CACHE_NAME);
+
+            assertNotNull(cache);
+
+            for (int i = 0; i < 100; i++) {
+                cache.put(i, node.name());
+
+                assertEquals(node.name(), cache.get(i));
+            }
+        }
+
+        awaitPartitionMapExchange();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartStop_2_Nodes() throws Exception {
+        ackEveryEventSystemProperty();
+
+        startGrid(0);
+
+        waitForTopology(1);
+
+        startGrid(1);
+
+        waitForTopology(2);
+
+        for (Ignite node : G.allGrids())
+            node.compute().broadcast(new DummyCallable(null));
+
+        awaitPartitionMapExchange();
+
+        waitForEventsAcks(ignite(0));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMultipleClusters() throws Exception {
+        Ignite c0 = startGrid(0);
+
+        zkRootPath = "/cluster2";
+
+        Ignite c1 = startGridsMultiThreaded(1, 5);
+
+        zkRootPath = "/cluster3";
+
+        Ignite c2 = startGridsMultiThreaded(6, 3);
+
+        checkNodesNumber(c0, 1);
+        checkNodesNumber(c1, 5);
+        checkNodesNumber(c2, 3);
+
+        stopGrid(2);
+
+        checkNodesNumber(c0, 1);
+        checkNodesNumber(c1, 4);
+        checkNodesNumber(c2, 3);
+
+        for (int i = 0; i < 3; i++)
+            stopGrid(i + 6);
+
+        checkNodesNumber(c0, 1);
+        checkNodesNumber(c1, 4);
+
+        c2 = startGridsMultiThreaded(6, 2);
+
+        checkNodesNumber(c0, 1);
+        checkNodesNumber(c1, 4);
+        checkNodesNumber(c2, 2);
+
+        evts.clear();
+    }
+
+    /**
+     * @param node Node.
+     * @param expNodes Expected node in cluster.
+     * @throws Exception If failed.
+     */
+    private void checkNodesNumber(final Ignite node, final int expNodes) throws Exception {
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return node.cluster().nodes().size() == expNodes;
+            }
+        }, 5000);
+
+        assertEquals(expNodes, node.cluster().nodes().size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartStop1() throws Exception {
+        ackEveryEventSystemProperty();
+
+        startGridsMultiThreaded(5, false);
+
+        waitForTopology(5);
+
+        awaitPartitionMapExchange();
+
+        waitForEventsAcks(ignite(0));
+
+        stopGrid(0);
+
+        waitForTopology(4);
+
+        for (Ignite node : G.allGrids())
+            node.compute().broadcast(new DummyCallable(null));
+
+        startGrid(0);
+
+        waitForTopology(5);
+
+        awaitPartitionMapExchange();
+
+        waitForEventsAcks(grid(CU.oldest(ignite(1).cluster().nodes())));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartStop3() throws Exception {
+        startGrids(4);
+
+        awaitPartitionMapExchange();
+
+        stopGrid(0);
+
+        startGrid(5);
+
+        awaitPartitionMapExchange();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartStop4() throws Exception {
+        startGrids(6);
+
+        awaitPartitionMapExchange();
+
+        stopGrid(2);
+
+        if (ThreadLocalRandom.current().nextBoolean())
+            awaitPartitionMapExchange();
+
+        stopGrid(1);
+
+        if (ThreadLocalRandom.current().nextBoolean())
+            awaitPartitionMapExchange();
+
+        stopGrid(0);
+
+        if (ThreadLocalRandom.current().nextBoolean())
+            awaitPartitionMapExchange();
+
+        startGrid(7);
+
+        awaitPartitionMapExchange();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartStop2() throws Exception {
+        startGridsMultiThreaded(10, false);
+
+        GridTestUtils.runMultiThreaded(new IgniteInClosure<Integer>() {
+            @Override public void apply(Integer idx) {
+                stopGrid(idx);
+            }
+        }, 3, "stop-node-thread");
+
+        waitForTopology(7);
+
+        startGridsMultiThreaded(0, 3);
+
+        waitForTopology(10);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartStopWithClients() throws Exception {
+        final int SRVS = 3;
+
+        startGrids(SRVS);
+
+        clientMode(true);
+
+        final int THREADS = 30;
+
+        for (int i = 0; i < 5; i++) {
+            info("Iteration: " + i);
+
+            startGridsMultiThreaded(SRVS, THREADS);
+
+            waitForTopology(SRVS + THREADS);
+
+            GridTestUtils.runMultiThreaded(new IgniteInClosure<Integer>() {
+                @Override public void apply(Integer idx) {
+                    stopGrid(idx + SRVS);
+                }
+            }, THREADS, "stop-node");
+
+            waitForTopology(SRVS);
+
+            checkEventsConsistency();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTopologyChangeMultithreaded() throws Exception {
+        topologyChangeWithRestarts(false, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTopologyChangeMultithreaded_RestartZk() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-8184");
+
+        try {
+            topologyChangeWithRestarts(true, false);
+        }
+        finally {
+            zkCluster.stop();
+
+            zkCluster = null;
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTopologyChangeMultithreaded_RestartZk_CloseClients() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-8184");
+
+        try {
+            topologyChangeWithRestarts(true, true);
+        }
+        finally {
+            zkCluster.stop();
+
+            zkCluster = null;
+        }
+    }
+
+    /**
+     * @param restartZk If {@code true} in background restarts on of ZK servers.
+     * @param closeClientSock If {@code true} in background closes zk clients' sockets.
+     * @throws Exception If failed.
+     */
+    private void topologyChangeWithRestarts(boolean restartZk, boolean closeClientSock) throws Exception {
+        sesTimeout = 30_000;
+
+        if (closeClientSock)
+            testSockNio = true;
+
+        long stopTime = System.currentTimeMillis() + 60_000;
+
+        AtomicBoolean stop = new AtomicBoolean();
+
+        IgniteInternalFuture<?> fut1 = null;
+
+        IgniteInternalFuture<?> fut2 = null;
+
+        try {
+            fut1 = restartZk ? startRestartZkServers(stopTime, stop) : null;
+            fut2 = closeClientSock ? startCloseZkClientSocket(stopTime, stop) : null;
+
+            int INIT_NODES = 10;
+
+            startGridsMultiThreaded(INIT_NODES);
+
+            final int MAX_NODES = 20;
+
+            final List<Integer> startedNodes = new ArrayList<>();
+
+            for (int i = 0; i < INIT_NODES; i++)
+                startedNodes.add(i);
+
+            ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+            final AtomicInteger startIdx = new AtomicInteger(INIT_NODES);
+
+            while (System.currentTimeMillis() < stopTime) {
+                if (startedNodes.size() >= MAX_NODES) {
+                    int stopNodes = rnd.nextInt(5) + 1;
+
+                    log.info("Next, stop nodes: " + stopNodes);
+
+                    final List<Integer> idxs = new ArrayList<>();
+
+                    while (idxs.size() < stopNodes) {
+                        Integer stopIdx = rnd.nextInt(startedNodes.size());
+
+                        if (!idxs.contains(stopIdx))
+                            idxs.add(startedNodes.get(stopIdx));
+                    }
+
+                    GridTestUtils.runMultiThreaded(new IgniteInClosure<Integer>() {
+                        @Override public void apply(Integer threadIdx) {
+                            int stopNodeIdx = idxs.get(threadIdx);
+
+                            info("Stop node: " + stopNodeIdx);
+
+                            stopGrid(stopNodeIdx);
+                        }
+                    }, stopNodes, "stop-node");
+
+                    startedNodes.removeAll(idxs);
+                }
+                else {
+                    int startNodes = rnd.nextInt(5) + 1;
+
+                    log.info("Next, start nodes: " + startNodes);
+
+                    GridTestUtils.runMultiThreaded(new Callable<Void>() {
+                        @Override public Void call() throws Exception {
+                            int idx = startIdx.incrementAndGet();
+
+                            log.info("Start node: " + idx);
+
+                            startGrid(idx);
+
+                            synchronized (startedNodes) {
+                                startedNodes.add(idx);
+                            }
+
+                            return null;
+                        }
+                    }, startNodes, "start-node");
+                }
+
+                U.sleep(rnd.nextInt(100) + 1);
+            }
+        }
+        finally {
+            stop.set(true);
+        }
+
+        if (fut1 != null)
+            fut1.get();
+
+        if (fut2 != null)
+            fut2.get();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRandomTopologyChanges() throws Exception {
+        randomTopologyChanges(false, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkZkNodesCleanup() throws Exception {
+        final ZookeeperClient zkClient = new ZookeeperClient(getTestResources().getLogger(),
+            zkCluster.getConnectString(),
+            30_000,
+            null);
+
+        final String basePath = IGNITE_ZK_ROOT + "/";
+
+        final String aliveDir = basePath + ZkIgnitePaths.ALIVE_NODES_DIR + "/";
+
+        try {
+            List<String> znodes = listSubTree(zkClient.zk(), IGNITE_ZK_ROOT);
+
+            boolean foundAlive = false;
+
+            for (String znode : znodes) {
+                if (znode.startsWith(aliveDir)) {
+                    foundAlive = true;
+
+                    break;
+                }
+            }
+
+            assertTrue(foundAlive); // Sanity check to make sure we check correct directory.
+
+            assertTrue("Failed to wait for unused znodes cleanup", GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                @Override public boolean apply() {
+                    try {
+                        List<String> znodes = listSubTree(zkClient.zk(), IGNITE_ZK_ROOT);
+
+                        for (String znode : znodes) {
+                            if (znode.startsWith(aliveDir) || znode.length() < basePath.length())
+                                continue;
+
+                            znode = znode.substring(basePath.length());
+
+                            if (!znode.contains("/")) // Ignore roots.
+                                continue;
+
+                            // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8193
+                            if (znode.startsWith("jd/"))
+                                continue;
+
+                            log.info("Found unexpected znode: " + znode);
+
+                            return false;
+                        }
+
+                        return true;
+                    }
+                    catch (Exception e) {
+                        error("Unexpected error: " + e, e);
+
+                        fail("Unexpected error: " + e);
+                    }
+
+                    return false;
+                }
+            }, 10_000));
+        }
+        finally {
+            zkClient.close();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRandomTopologyChanges_RestartZk() throws Exception {
+        randomTopologyChanges(true, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRandomTopologyChanges_CloseClients() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-8182");
+
+        randomTopologyChanges(false, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeployService1() throws Exception {
+        startGridsMultiThreaded(3);
+
+        grid(0).services(grid(0).cluster()).deployNodeSingleton("test", new GridCacheAbstractFullApiSelfTest.DummyServiceImpl());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeployService2() throws Exception {
+        clientMode(false);
+
+        startGrid(0);
+
+        clientMode(true);
+
+        startGrid(1);
+
+        grid(0).services(grid(0).cluster()).deployNodeSingleton("test", new GridCacheAbstractFullApiSelfTest.DummyServiceImpl());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeployService3() throws Exception {
+        IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() {
+            @Override public Object call() throws Exception {
+                clientModeThreadLocal(true);
+
+                startGrid(0);
+
+                return null;
+            }
+        }, "start-node");
+
+        clientModeThreadLocal(false);
+
+        startGrid(1);
+
+        fut.get();
+
+        grid(0).services(grid(0).cluster()).deployNodeSingleton("test", new GridCacheAbstractFullApiSelfTest.DummyServiceImpl());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLargeUserAttribute1() throws Exception {
+        initLargeAttribute();
+
+        startGrid(0);
+
+        checkZkNodesCleanup();
+
+        userAttrs = null;
+
+        startGrid(1);
+
+        waitForEventsAcks(ignite(0));
+
+        waitForTopology(2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLargeUserAttribute2() throws Exception {
+        startGrid(0);
+
+        initLargeAttribute();
+
+        startGrid(1);
+
+        waitForEventsAcks(ignite(0));
+
+        checkZkNodesCleanup();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLargeUserAttribute3() throws Exception {
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        long stopTime = System.currentTimeMillis() + 60_000;
+
+        int nodes = 0;
+
+        for (int i = 0; i < 25; i++) {
+            info("Iteration: " + i);
+
+            if (rnd.nextBoolean())
+                initLargeAttribute();
+            else
+                userAttrs = null;
+
+            clientMode(i > 5);
+
+            startGrid(i);
+
+            nodes++;
+
+            if (System.currentTimeMillis() >= stopTime)
+                break;
+        }
+
+        waitForTopology(nodes);
+    }
+
+    /**
+     *
+     */
+    private void initLargeAttribute() {
+        userAttrs = new HashMap<>();
+
+        int[] attr = new int[1024 * 1024 + ThreadLocalRandom.current().nextInt(1024)];
+
+        for (int i = 0; i < attr.length; i++)
+            attr[i] = i;
+
+        userAttrs.put("testAttr", attr);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLargeCustomEvent() throws Exception {
+        Ignite srv0 = startGrid(0);
+
+        // Send large message, single node in topology.
+        IgniteCache<Object, Object> cache = srv0.createCache(largeCacheConfiguration("c1"));
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, i);
+
+        assertEquals(1, cache.get(1));
+
+        waitForEventsAcks(ignite(0));
+
+        startGridsMultiThreaded(1, 3);
+
+        srv0.destroyCache("c1");
+
+        // Send large message, multiple nodes in topology.
+        cache = srv0.createCache(largeCacheConfiguration("c1"));
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, i);
+
+        waitForTopology(4);
+
+        ignite(3).createCache(largeCacheConfiguration("c2"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnectSessionExpire1_1() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-8131");
+
+        clientReconnectSessionExpire(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnectSessionExpire1_2() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-8131");
+
+        clientReconnectSessionExpire(true);
+    }
+
+    /**
+     * @param closeSock Test mode flag.
+     * @throws Exception If failed.
+     */
+    private void clientReconnectSessionExpire(boolean closeSock) throws Exception {
+        startGrid(0);
+
+        sesTimeout = 2000;
+        clientMode(true);
+        testSockNio = true;
+
+        Ignite client = startGrid(1);
+
+        client.cache(DEFAULT_CACHE_NAME).put(1, 1);
+
+        reconnectClientNodes(log, Collections.singletonList(client), closeSock);
+
+        assertEquals(1, client.cache(DEFAULT_CACHE_NAME).get(1));
+
+        client.compute().broadcast(new DummyCallable(null));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testForceClientReconnect() throws Exception {
+        final int SRVS = 3;
+
+        startGrids(SRVS);
+
+        clientMode(true);
+
+        startGrid(SRVS);
+
+        reconnectClientNodes(Collections.singletonList(ignite(SRVS)), new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                ZookeeperDiscoverySpi spi = waitSpi(getTestIgniteInstanceName(SRVS));
+
+                spi.clientReconnect();
+
+                return null;
+            }
+        });
+
+        waitForTopology(SRVS + 1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testForcibleClientFail() throws Exception {
+        final int SRVS = 3;
+
+        startGrids(SRVS);
+
+        clientMode(true);
+
+        startGrid(SRVS);
+
+        reconnectClientNodes(Collections.singletonList(ignite(SRVS)), new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                ZookeeperDiscoverySpi spi = waitSpi(getTestIgniteInstanceName(0));
+
+                spi.failNode(ignite(SRVS).cluster().localNode().id(), "Test forcible node fail");
+
+                return null;
+            }
+        });
+
+        waitForTopology(SRVS + 1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDuplicatedNodeId() throws Exception {
+        UUID nodeId0 = nodeId = UUID.randomUUID();
+
+        startGrid(0);
+
+        int failingNodeIdx = 100;
+
+        for (int i = 0; i < 5; i++) {
+            final int idx = failingNodeIdx++;
+
+            nodeId = nodeId0;
+
+            info("Start node with duplicated ID [iter=" + i + ", nodeId=" + nodeId + ']');
+
+            Throwable err = GridTestUtils.assertThrows(log, new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    startGrid(idx);
+
+                    return null;
+                }
+            }, IgniteCheckedException.class, null);
+
+            IgniteSpiException spiErr = X.cause(err, IgniteSpiException.class);
+
+            assertNotNull(spiErr);
+            assertTrue(spiErr.getMessage().contains("Node with the same ID already exists"));
+
+            nodeId = null;
+
+            info("Start node with unique ID [iter=" + i + ']');
+
+            Ignite ignite = startGrid(idx);
+
+            nodeId0 = ignite.cluster().localNode().id();
+
+            waitForTopology(i + 2);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPing() throws Exception {
+        sesTimeout = 5000;
+
+        startGrids(3);
+
+        final ZookeeperDiscoverySpi spi = waitSpi(getTestIgniteInstanceName(1));
+
+        final UUID nodeId = ignite(2).cluster().localNode().id();
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+            @Override public void run() {
+                assertTrue(spi.pingNode(nodeId));
+            }
+        }, 32, "ping");
+
+        fut.get();
+
+        fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+            @Override public void run() {
+                spi.pingNode(nodeId);
+            }
+        }, 32, "ping");
+
+        U.sleep(100);
+
+        stopGrid(2);
+
+        fut.get();
+
+        fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+            @Override public void run() {
+                assertFalse(spi.pingNode(nodeId));
+            }
+        }, 32, "ping");
+
+        fut.get();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testWithPersistence1() throws Exception {
+        startWithPersistence(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testWithPersistence2() throws Exception {
+        startWithPersistence(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoOpCommunicationFailureResolve_1() throws Exception {
+        communicationFailureResolve_Simple(2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoOpCommunicationErrorResolve_2() throws Exception {
+        communicationFailureResolve_Simple(10);
+    }
+
+    /**
+     * @param nodes Nodes number.
+     * @throws Exception If failed.
+     */
+    private void communicationFailureResolve_Simple(int nodes) throws Exception {
+        assert nodes > 1;
+
+        sesTimeout = 2000;
+        commFailureRslvr = NoOpCommunicationFailureResolver.FACTORY;
+
+        startGridsMultiThreaded(nodes);
+
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        for (int i = 0; i < 3; i++) {
+            info("Iteration: " + i);
+
+            int idx1 = rnd.nextInt(nodes);
+
+            int idx2;
+
+            do {
+                idx2 = rnd.nextInt(nodes);
+            }
+            while (idx1 == idx2);
+
+            ZookeeperDiscoverySpi spi = spi(ignite(idx1));
+
+            spi.resolveCommunicationFailure(ignite(idx2).cluster().localNode(), new Exception("test"));
+
+            checkInternalStructuresCleanup();
+        }
+    }
+
+    /**
+     * Tests case when one node fails before sending communication status.
+     *
+     * @throws Exception If failed.
+     */
+    public void testNoOpCommunicationErrorResolve_3() throws Exception {
+        sesTimeout = 2000;
+        commFailureRslvr = NoOpCommunicationFailureResolver.FACTORY;
+
+        startGridsMultiThreaded(3);
+
+        sesTimeout = 10_000;
+
+        testSockNio = true;
+        sesTimeout = 5000;
+
+        startGrid(3);
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() {
+                ZookeeperDiscoverySpi spi = spi(ignite(0));
+
+                spi.resolveCommunicationFailure(ignite(1).cluster().localNode(), new Exception("test"));
+
+                return null;
+            }
+        });
+
+        U.sleep(1000);
+
+        ZkTestClientCnxnSocketNIO nio = ZkTestClientCnxnSocketNIO.forNode(ignite(3));
+
+        nio.closeSocket(true);
+
+        try {
+            stopGrid(3);
+
+            fut.get();
+        }
+        finally {
+            nio.allowConnect();
+        }
+
+        waitForTopology(3);
+    }
+
+    /**
+     * Tests case when Coordinator fails while resolve process is in progress.
+     *
+     * @throws Exception If failed.
+     */
+    public void testNoOpCommunicationErrorResolve_4() throws Exception {
+        testCommSpi = true;
+
+        sesTimeout = 2000;
+        commFailureRslvr = NoOpCommunicationFailureResolver.FACTORY;
+
+        startGrid(0);
+
+        startGridsMultiThreaded(1, 3);
+
+        ZkTestCommunicationSpi commSpi = ZkTestCommunicationSpi.testSpi(ignite(3));
+
+        commSpi.pingLatch = new CountDownLatch(1);
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() {
+                ZookeeperDiscoverySpi spi = spi(ignite(1));
+
+                spi.resolveCommunicationFailure(ignite(2).cluster().localNode(), new Exception("test"));
+
+                return null;
+            }
+        });
+
+        U.sleep(1000);
+
+        assertFalse(fut.isDone());
+
+        stopGrid(0);
+
+        commSpi.pingLatch.countDown();
+
+        fut.get();
+
+        waitForTopology(3);
+    }
+
+    /**
+     * Tests that nodes join is delayed while resolve is in progress.
+     *
+     * @throws Exception If failed.
+     */
+    public void testNoOpCommunicationErrorResolve_5() throws Exception {
+        testCommSpi = true;
+
+        sesTimeout = 2000;
+        commFailureRslvr = NoOpCommunicationFailureResolver.FACTORY;
+
+        startGrid(0);
+
+        startGridsMultiThreaded(1, 3);
+
+        ZkTestCommunicationSpi commSpi = ZkTestCommunicationSpi.testSpi(ignite(3));
+
+        commSpi.pingStartLatch = new CountDownLatch(1);
+        commSpi.pingLatch = new CountDownLatch(1);
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() {
+                ZookeeperDiscoverySpi spi = spi(ignite(1));
+
+                spi.resolveCommunicationFailure(ignite(2).cluster().localNode(), new Exception("test"));
+
+                return null;
+            }
+        });
+
+        assertTrue(commSpi.pingStartLatch.await(10, SECONDS));
+
+        try {
+            assertFalse(fut.isDone());
+
+            final AtomicInteger nodeIdx = new AtomicInteger(3);
+
+            IgniteInternalFuture<?> startFut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    startGrid(nodeIdx.incrementAndGet());
+
+                    return null;
+                }
+            }, 3, "start-node");
+
+            U.sleep(1000);
+
+            assertFalse(startFut.isDone());
+
+            assertEquals(4, ignite(0).cluster().nodes().size());
+
+            commSpi.pingLatch.countDown();
+
+            startFut.get();
+            fut.get();
+
+            waitForTopology(7);
+        }
+        finally {
+            commSpi.pingLatch.countDown();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCommunicationErrorResolve_KillNode_1() throws Exception {
+        communicationFailureResolve_KillNodes(2, Collections.singleton(2L));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCommunicationErrorResolve_KillNode_2() throws Exception {
+        communicationFailureResolve_KillNodes(3, Collections.singleton(2L));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCommunicationErrorResolve_KillNode_3() throws Exception {
+        communicationFailureResolve_KillNodes(10, Arrays.asList(2L, 4L, 6L));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCommunicationErrorResolve_KillCoordinator_1() throws Exception {
+        communicationFailureResolve_KillNodes(2, Collections.singleton(1L));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCommunicationErrorResolve_KillCoordinator_2() throws Exception {
+        communicationFailureResolve_KillNodes(3, Collections.singleton(1L));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCommunicationErrorResolve_KillCoordinator_3() throws Exception {
+        communicationFailureResolve_KillNodes(10, Arrays.asList(1L, 4L, 6L));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCommunicationErrorResolve_KillCoordinator_4() throws Exception {
+        communicationFailureResolve_KillNodes(10, Arrays.asList(1L, 2L, 3L));
+    }
+
+    /**
+     * @param startNodes Number of nodes to start.
+     * @param killNodes Nodes to kill by resolve process.
+     * @throws Exception If failed.
+     */
+    private void communicationFailureResolve_KillNodes(int startNodes, Collection<Long> killNodes) throws Exception {
+        testCommSpi = true;
+
+        commFailureRslvr = TestNodeKillCommunicationFailureResolver.factory(killNodes);
+
+        startGrids(startNodes);
+
+        ZkTestCommunicationSpi commSpi = ZkTestCommunicationSpi.testSpi(ignite(0));
+
+        commSpi.checkRes = new BitSet(startNodes);
+
+        ZookeeperDiscoverySpi spi = null;
+        UUID killNodeId = null;
+
+        for (Ignite node : G.allGrids()) {
+            ZookeeperDiscoverySpi spi0 = spi(node);
+
+            if (!killNodes.contains(node.cluster().localNode().order()))
+                spi = spi0;
+            else
+                killNodeId = node.cluster().localNode().id();
+        }
+
+        assertNotNull(spi);
+        assertNotNull(killNodeId);
+
+        try {
+            spi.resolveCommunicationFailure(spi.getNode(killNodeId), new Exception("test"));
+
+            fail("Exception is not thrown");
+        }
+        catch (IgniteSpiException e) {
+            assertTrue("Unexpected exception: " + e, e.getCause() instanceof ClusterTopologyCheckedException);
+        }
+
+        int expNodes = startNodes - killNodes.size();
+
+        waitForTopology(expNodes);
+
+        for (Ignite node : G.allGrids())
+            assertFalse(killNodes.contains(node.cluster().localNode().order()));
+
+        startGrid(startNodes);
+
+        waitForTopology(expNodes + 1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCommunicationFailureResolve_KillCoordinator_5() throws Exception {
+        sesTimeout = 2000;
+
+        testCommSpi = true;
+        commFailureRslvr = KillCoordinatorCommunicationFailureResolver.FACTORY;
+
+        startGrids(10);
+
+        int crd = 0;
+
+        int nodeIdx = 10;
+
+        for (int i = 0; i < 10; i++) {
+            info("Iteration: " + i);
+
+            for (Ignite node : G.allGrids())
+                ZkTestCommunicationSpi.testSpi(node).initCheckResult(10);
+
+            UUID crdId = ignite(crd).cluster().localNode().id();
+
+            ZookeeperDiscoverySpi spi = spi(ignite(crd + 1));
+
+            try {
+                spi.resolveCommunicationFailure(spi.getNode(crdId), new Exception("test"));
+
+                fail("Exception is not thrown");
+            }
+            catch (IgniteSpiException e) {
+                assertTrue("Unexpected exception: " + e, e.getCause() instanceof ClusterTopologyCheckedException);
+            }
+
+            waitForTopology(9);
+
+            startGrid(nodeIdx++);
+
+            waitForTopology(10);
+
+            crd++;
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCommunicationFailureResolve_KillRandom() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-8179");
+
+        sesTimeout = 2000;
+
+        testCommSpi = true;
+        commFailureRslvr = KillRandomCommunicationFailureResolver.FACTORY;
+
+        startGridsMultiThreaded(10);
+
+        clientMode(true);
+
+        startGridsMultiThreaded(10, 5);
+
+        int nodeIdx = 15;
+
+        for (int i = 0; i < 10; i++) {
+            info("Iteration: " + i);
+
+            ZookeeperDiscoverySpi spi = null;
+
+            for (Ignite node : G.allGrids()) {
+                ZkTestCommunicationSpi.testSpi(node).initCheckResult(100);
+
+                spi = spi(node);
+            }
+
+            assert spi != null;
+
+            try {
+                spi.resolveCommunicationFailure(spi.getRemoteNodes().iterator().next(), new Exception("test"));
+            }
+            catch (IgniteSpiException ignore) {
+                // No-op.
+            }
+
+            clientMode(ThreadLocalRandom.current().nextBoolean());
+
+            startGrid(nodeIdx++);
+
+            awaitPartitionMapExchange();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDefaultCommunicationFailureResolver1() throws Exception {
+        testCommSpi = true;
+        sesTimeout = 5000;
+
+        startGrids(3);
+
+        ZkTestCommunicationSpi.testSpi(ignite(0)).initCheckResult(3, 0, 1);
+        ZkTestCommunicationSpi.testSpi(ignite(1)).initCheckResult(3, 0, 1);
+        ZkTestCommunicationSpi.testSpi(ignite(2)).initCheckResult(3, 2);
+
+        UUID killedId = nodeId(2);
+
+        assertNotNull(ignite(0).cluster().node(killedId));
+
+        ZookeeperDiscoverySpi spi = spi(ignite(0));
+
+        spi.resolveCommunicationFailure(spi.getNode(ignite(1).cluster().localNode().id()), new Exception("test"));
+
+        waitForTopology(2);
+
+        assertNull(ignite(0).cluster().node(killedId));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDefaultCommunicationFailureResolver2() throws Exception {
+        testCommSpi = true;
+        sesTimeout = 5000;
+
+        startGrids(3);
+
+        clientMode(true);
+
+        startGridsMultiThreaded(3, 2);
+
+        ZkTestCommunicationSpi.testSpi(ignite(0)).initCheckResult(5, 0, 1);
+        ZkTestCommunicationSpi.testSpi(ignite(1)).initCheckResult(5, 0, 1);
+        ZkTestCommunicationSpi.testSpi(ignite(2)).initCheckResult(5, 2, 3, 4);
+        ZkTestCommunicationSpi.testSpi(ignite(3)).initCheckResult(5, 2, 3, 4);
+        ZkTestCommunicationSpi.testSpi(ignite(4)).initCheckResult(5, 2, 3, 4);
+
+        ZookeeperDiscoverySpi spi = spi(ignite(0));
+
+        spi.resolveCommunicationFailure(spi.getNode(ignite(1).cluster().localNode().id()), new Exception("test"));
+
+        waitForTopology(2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDefaultCommunicationFailureResolver3() throws Exception {
+        defaultCommunicationFailureResolver_BreakCommunication(3, 1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDefaultCommunicationFailureResolver4() throws Exception {
+        defaultCommunicationFailureResolver_BreakCommunication(3, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDefaultCommunicationFailureResolver5() throws Exception {
+        defaultCommunicationFailureResolver_BreakCommunication(10, 1, 3, 6);
+    }
+
+    /**
+     * @param startNodes Initial nodes number.
+     * @param breakNodes Node indices where communication server is closed.
+     * @throws Exception If failed.
+     */
+    private void defaultCommunicationFailureResolver_BreakCommunication(int startNodes, final int...breakNodes) throws Exception {
+        sesTimeout = 5000;
+
+        startGridsMultiThreaded(startNodes);
+
+        final CyclicBarrier b = new CyclicBarrier(breakNodes.length);
+
+        GridTestUtils.runMultiThreaded(new IgniteInClosure<Integer>() {
+            @Override public void apply(Integer threadIdx) {
+                try {
+                    b.await();
+
+                    int nodeIdx = breakNodes[threadIdx];
+
+                    info("Close communication: " + nodeIdx);
+
+                    ((TcpCommunicationSpi)ignite(nodeIdx).configuration().getCommunicationSpi()).simulateNodeFailure();
+                }
+                catch (Exception e) {
+                    fail("Unexpected error: " + e);
+                }
+            }
+        }, breakNodes.length, "break-communication");
+
+        waitForTopology(startNodes - breakNodes.length);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCommunicationFailureResolve_CachesInfo1() throws Exception {
+        testCommSpi = true;
+        sesTimeout = 5000;
+
+        final CacheInfoCommunicationFailureResolver rslvr = new CacheInfoCommunicationFailureResolver();
+
+        commFailureRslvr = new IgniteOutClosure<CommunicationFailureResolver>() {
+            @Override public CommunicationFailureResolver apply() {
+                return rslvr;
+            }
+        };
+
+        startGrids(2);
+
+        awaitPartitionMapExchange();
+
+        Map<String, T3<Integer, Integer, Integer>> expCaches = new HashMap<>();
+
+        expCaches.put(DEFAULT_CACHE_NAME, new T3<>(RendezvousAffinityFunction.DFLT_PARTITION_COUNT, 0, 1));
+
+        checkResolverCachesInfo(ignite(0), expCaches);
+
+        List<CacheConfiguration> caches = new ArrayList<>();
+
+        CacheConfiguration c1 = new CacheConfiguration("c1");
+        c1.setBackups(1);
+        c1.setAffinity(new RendezvousAffinityFunction(false, 64));
+        caches.add(c1);
+
+        CacheConfiguration c2 = new CacheConfiguration("c2");
+        c2.setBackups(2);
+        c2.setAffinity(new RendezvousAffinityFunction(false, 128));
+        caches.add(c2);
+
+        CacheConfiguration c3 = new CacheConfiguration("c3");
+        c3.setCacheMode(CacheMode.REPLICATED);
+        c3.setAffinity(new RendezvousAffinityFunction(false, 256));
+        caches.add(c3);
+
+        ignite(0).createCaches(caches);
+
+        expCaches.put("c1", new T3<>(64, 1, 2));
+        expCaches.put("c2", new T3<>(128, 2, 2));
+        expCaches.put("c3", new T3<>(256, 1, 2));
+
+        checkResolverCachesInfo(ignite(0), expCaches);
+
+        startGrid(2);
+        startGrid(3);
+
+        awaitPartitionMapExchange();
+
+        expCaches.put("c2", new T3<>(128, 2, 3));
+        expCaches.put("c3", new T3<>(256, 1, 4));
+
+        checkResolverCachesInfo(ignite(0), expCaches);
+
+        CacheConfiguration c4 = new CacheConfiguration("c4");
+        c4.setCacheMode(CacheMode.PARTITIONED);
+        c4.setBackups(0);
+        c4.setAffinity(new RendezvousAffinityFunction(false, 256));
+        c4.setNodeFilter(new TestCacheNodeExcludingFilter(getTestIgniteInstanceName(0), getTestIgniteInstanceName(1)));
+
+        ignite(2).createCache(c4);
+
+        expCaches.put("c4", new T3<>(256, 0, 1));
+
+        checkResolverCachesInfo(ignite(0), expCaches);
+
+        stopGrid(0); // Stop current coordinator, check new coordinator will initialize required caches information.
+
+        awaitPartitionMapExchange();
+
+        expCaches.put("c3", new T3<>(256, 1, 3));
+
+        checkResolverCachesInfo(ignite(1), expCaches);
+
+        startGrid(0);
+
+        expCaches.put("c3", new T3<>(256, 1, 4));
+
+        checkResolverCachesInfo(ignite(1), expCaches);
+
+        stopGrid(1);
+
+        expCaches.put("c3", new T3<>(256, 1, 3));
+
+        checkResolverCachesInfo(ignite(3), expCaches);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCommunicationFailureResolve_CachesInfo2() throws Exception {
+        testCommSpi = true;
+        sesTimeout = 5000;
+
+        final CacheInfoCommunicationFailureResolver rslvr = new CacheInfoCommunicationFailureResolver();
+
+        commFailureRslvr = new IgniteOutClosure<CommunicationFailureResolver>() {
+            @Override public CommunicationFailureResolver apply() {
+                return rslvr;
+            }
+        };
+
+        Ignite srv0 = startGrid(0);
+
+        CacheConfiguration ccfg = new CacheConfiguration("c1

<TRUNCATED>

[11/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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 9402a32..38450df 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
@@ -155,22 +155,30 @@ final class BinaryMetadataTransport {
      * @param metadata Metadata proposed for update.
      * @return Future to wait for update result on.
      */
-    GridFutureAdapter<MetadataUpdateResult> requestMetadataUpdate(BinaryMetadata metadata) throws IgniteCheckedException {
+    GridFutureAdapter<MetadataUpdateResult> requestMetadataUpdate(BinaryMetadata metadata) {
         MetadataUpdateResultFuture resFut = new MetadataUpdateResultFuture();
 
         if (log.isDebugEnabled())
             log.debug("Requesting metadata update for " + metadata.typeId() + "; caller thread is blocked on future "
                 + resFut);
 
-        synchronized (this) {
-            unlabeledFutures.add(resFut);
+        try {
+            synchronized (this) {
+                unlabeledFutures.add(resFut);
 
-            if (!stopping)
-                discoMgr.sendCustomEvent(new MetadataUpdateProposedMessage(metadata, ctx.localNodeId()));
-            else
-                resFut.onDone(MetadataUpdateResult.createUpdateDisabledResult());
+                if (!stopping)
+                    discoMgr.sendCustomEvent(new MetadataUpdateProposedMessage(metadata, ctx.localNodeId()));
+                else
+                    resFut.onDone(MetadataUpdateResult.createUpdateDisabledResult());
+            }
+        }
+        catch (Exception e) {
+            resFut.onDone(MetadataUpdateResult.createUpdateDisabledResult(), e);
         }
 
+        if (ctx.clientDisconnected())
+            onDisconnected();
+
         return resFut;
     }
 
@@ -237,6 +245,8 @@ final class BinaryMetadataTransport {
         for (MetadataUpdateResultFuture fut : unlabeledFutures)
             fut.onDone(res);
 
+        unlabeledFutures.clear();
+
         for (MetadataUpdateResultFuture fut : syncMap.values())
             fut.onDone(res);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateAcceptedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateAcceptedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateAcceptedMessage.java
index 0416746..df64613 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateAcceptedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateAcceptedMessage.java
@@ -71,6 +71,11 @@ public class MetadataUpdateAcceptedMessage implements DiscoveryCustomMessage {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
         AffinityTopologyVersion topVer, DiscoCache discoCache) {
         throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateProposedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateProposedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateProposedMessage.java
index f9bd660..84e32e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateProposedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateProposedMessage.java
@@ -134,6 +134,11 @@ public final class MetadataUpdateProposedMessage implements DiscoveryCustomMessa
     }
 
     /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
         AffinityTopologyVersion topVer, DiscoCache discoCache) {
         throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
index 9b3c1ec..5bbbb31 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
@@ -598,6 +598,45 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
+    @Override public List<List<ClusterNode>> allOwners() {
+        lock.readLock().lock();
+
+        try {
+            int parts = partitions();
+
+            List<List<ClusterNode>> res = new ArrayList<>(parts);
+
+            for (int i = 0; i < parts; i++)
+                res.add(new ArrayList<>());
+
+            List<ClusterNode> allNodes = discoCache.cacheGroupAffinityNodes(grpId);
+
+            for (int i = 0; i < allNodes.size(); i++) {
+                ClusterNode node = allNodes.get(i);
+
+                GridDhtPartitionMap nodeParts = node2part.get(node.id());
+
+                if (nodeParts != null) {
+                    for (Map.Entry<Integer, GridDhtPartitionState> e : nodeParts.map().entrySet()) {
+                        if (e.getValue() == OWNING) {
+                            int part = e.getKey();
+
+                            List<ClusterNode> owners = res.get(part);
+
+                            owners.add(node);
+                        }
+                    }
+                }
+            }
+
+            return res;
+        }
+        finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public List<ClusterNode> moving(int p) {
         return nodes(p, AffinityTopologyVersion.NONE, MOVING, null);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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 ba55543..ea99f5d 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
@@ -970,6 +970,10 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
                 try {
                     ctx.io().send(nodeId, res, ctx.ioPolicy());
                 }
+                catch (ClusterTopologyCheckedException e) {
+                    if (log.isDebugEnabled())
+                        log.debug("Failed to send get response to node, node failed: " + nodeId);
+                }
                 catch (IgniteCheckedException e) {
                     U.error(log, "Failed to send get response to node (is node still alive?) [nodeId=" + nodeId +
                         ",req=" + req + ", res=" + res + ']', e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
index 13564c2..7f900cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
@@ -237,6 +237,12 @@ public interface GridDhtPartitionTopology {
     public List<ClusterNode> owners(int p);
 
     /**
+     * @return List indexed by partition number, each list element is collection of all nodes who
+     *      owns corresponding partition.
+     */
+    public List<List<ClusterNode>> allOwners();
+
+    /**
      * @param p Partition ID.
      * @param topVer Topology version.
      * @return Collection of all nodes who {@code own} this partition.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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 528f0a6..538c57e 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
@@ -1217,6 +1217,45 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
+    @Override public List<List<ClusterNode>> allOwners() {
+        lock.readLock().lock();
+
+        try {
+            int parts = partitions();
+
+            List<List<ClusterNode>> res = new ArrayList<>(parts);
+
+            for (int i = 0; i < parts; i++)
+                res.add(new ArrayList<>());
+
+            List<ClusterNode> allNodes = discoCache.cacheGroupAffinityNodes(grp.groupId());
+
+            for (int i = 0; i < allNodes.size(); i++) {
+                ClusterNode node = allNodes.get(i);
+
+                GridDhtPartitionMap nodeParts = node2part.get(node.id());
+
+                if (nodeParts != null) {
+                    for (Map.Entry<Integer, GridDhtPartitionState> e : nodeParts.map().entrySet()) {
+                        if (e.getValue() == OWNING) {
+                            int part = e.getKey();
+
+                            List<ClusterNode> owners = res.get(part);
+
+                            owners.add(node);
+                        }
+                    }
+                }
+            }
+
+            return res;
+        }
+        finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public List<ClusterNode> moving(int p) {
         if (!grp.rebalanceEnabled())
             return ownersAndMoving(p, AffinityTopologyVersion.NONE);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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 8da91a8..cbb4985 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
@@ -1506,12 +1506,16 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         }
         catch (ClusterTopologyCheckedException ignore) {
             if (log.isDebugEnabled())
-                log.debug("Oldest node left during partition exchange [nodeId=" + oldestNode.id() +
+                log.debug("Coordinator left during partition exchange [nodeId=" + oldestNode.id() +
                     ", exchId=" + exchId + ']');
         }
         catch (IgniteCheckedException e) {
-            U.error(log, "Failed to send local partitions to oldest node (will retry after timeout) [oldestNodeId=" +
-                oldestNode.id() + ", exchId=" + exchId + ']', e);
+            if (reconnectOnError(e))
+                onDone(new IgniteNeedReconnectException(cctx.localNode(), e));
+            else {
+                U.error(log, "Failed to send local partitions to coordinator [crd=" + oldestNode.id() +
+                    ", exchId=" + exchId + ']', e);
+            }
         }
     }
 
@@ -3369,9 +3373,13 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                             }
 
                             if (allReceived) {
-                                awaitSingleMapUpdates();
+                                cctx.kernalContext().getSystemExecutorService().submit(new Runnable() {
+                                    @Override public void run() {
+                                        awaitSingleMapUpdates();
 
-                                onAllReceived(null);
+                                        onAllReceived(null);
+                                    }
+                                });
                             }
                         }
                         else {
@@ -3399,7 +3407,13 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                                         ", newCrd=" + crd0.id() + ']');
                                 }
 
-                                sendPartitions(crd0);
+                                final ClusterNode newCrd = crd0;
+
+                                cctx.kernalContext().getSystemExecutorService().submit(new Runnable() {
+                                    @Override public void run() {
+                                        sendPartitions(newCrd);
+                                    }
+                                });
                             }
                         }
                     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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
index d7dfa16..bbbd999 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java
@@ -94,6 +94,11 @@ public class ChangeGlobalStateFinishMessage implements DiscoveryCustomMessage {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
         AffinityTopologyVersion topVer, DiscoCache discoCache) {
         throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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
index 50fc022..81855fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java
@@ -131,6 +131,11 @@ public class ChangeGlobalStateMessage implements DiscoveryCustomMessage {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer,
         DiscoCache discoCache) {
         return mgr.createDiscoCacheOnCacheChange(topVer, discoCache);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterMetricsUpdateMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterMetricsUpdateMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterMetricsUpdateMessage.java
new file mode 100644
index 0000000..5c3044b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterMetricsUpdateMessage.java
@@ -0,0 +1,158 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.internal.GridDirectMap;
+import org.apache.ignite.internal.managers.communication.GridIoMessageFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+public class ClusterMetricsUpdateMessage implements Message {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private byte[] nodeMetrics;
+
+    /** */
+    @GridDirectMap(keyType = UUID.class, valueType = byte[].class)
+    private Map<UUID, byte[]> allNodesMetrics;
+
+    /**
+     * Required by {@link GridIoMessageFactory}.
+     */
+    public ClusterMetricsUpdateMessage() {
+        // No-op.
+    }
+
+    /**
+     * @param nodeMetrics Node metrics.
+     */
+    ClusterMetricsUpdateMessage(byte[] nodeMetrics) {
+        this.nodeMetrics = nodeMetrics;
+    }
+
+    /**
+     * @param allNodesMetrics All nodes metrcis.
+     */
+    ClusterMetricsUpdateMessage(Map<UUID, byte[]> allNodesMetrics) {
+        this.allNodesMetrics = allNodesMetrics;
+    }
+
+    /**
+     * @return Node metrics.
+     */
+    @Nullable byte[] nodeMetrics() {
+        return nodeMetrics;
+    }
+
+    /**
+     * @return All nodes metrics.
+     */
+    @Nullable Map<UUID, byte[]> allNodesMetrics() {
+        return allNodesMetrics;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeMap("allNodesMetrics", allNodesMetrics, MessageCollectionItemType.UUID, MessageCollectionItemType.BYTE_ARR))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeByteArray("nodeMetrics", nodeMetrics))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                allNodesMetrics = reader.readMap("allNodesMetrics", MessageCollectionItemType.UUID, MessageCollectionItemType.BYTE_ARR, false);
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                nodeMetrics = reader.readByteArray("nodeMetrics");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(ClusterMetricsUpdateMessage.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public short directType() {
+        return 133;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 2;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(ClusterMetricsUpdateMessage.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterNodeMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterNodeMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterNodeMetrics.java
new file mode 100644
index 0000000..22a385f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterNodeMetrics.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cluster;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Map;
+import org.apache.ignite.cache.CacheMetrics;
+import org.apache.ignite.cluster.ClusterMetrics;
+import org.apache.ignite.internal.ClusterMetricsSnapshot;
+
+/**
+ *
+ */
+class ClusterNodeMetrics implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final byte[] metrics;
+
+    /** */
+    private final Map<Integer, CacheMetrics> cacheMetrics;
+
+    /**
+     * @param metrics Metrics.
+     * @param cacheMetrics Cache metrics.
+     */
+    ClusterNodeMetrics(ClusterMetrics metrics, Map<Integer, CacheMetrics> cacheMetrics) {
+        this.metrics = ClusterMetricsSnapshot.serialize(metrics);
+        this.cacheMetrics = cacheMetrics;
+    }
+
+    /**
+     * @return Metrics.
+     */
+    byte[] metrics() {
+        return metrics;
+    }
+
+    /**
+     * @return Cache metrics.
+     */
+    Map<Integer, CacheMetrics> cacheMetrics() {
+        return cacheMetrics != null ? cacheMetrics : Collections.<Integer, CacheMetrics>emptyMap();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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 5f2c66c..8796302 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
@@ -33,6 +33,8 @@ import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.Event;
+import org.apache.ignite.internal.ClusterMetricsSnapshot;
+import org.apache.ignite.internal.GridDirectMap;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteDiagnosticInfo;
 import org.apache.ignite.internal.IgniteDiagnosticMessage;
@@ -42,21 +44,29 @@ import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.cluster.IgniteClusterImpl;
 import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.managers.communication.GridMessageListener;
+import org.apache.ignite.internal.managers.discovery.DiscoCache;
+import org.apache.ignite.internal.managers.discovery.IgniteClusterNode;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
 import org.apache.ignite.internal.util.GridTimerTask;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.CI1;
+import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.marshaller.jdk.JdkMarshaller;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag.GridDiscoveryData;
+import org.apache.ignite.spi.discovery.DiscoveryMetricsProvider;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_DIAGNOSTIC_ENABLED;
@@ -66,6 +76,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.CLUSTER_PROC;
 import static org.apache.ignite.internal.GridTopic.TOPIC_INTERNAL_DIAGNOSTIC;
+import static org.apache.ignite.internal.GridTopic.TOPIC_METRICS;
 import static org.apache.ignite.internal.IgniteVersionUtils.VER_STR;
 
 /**
@@ -102,6 +113,18 @@ public class ClusterProcessor extends GridProcessorAdapter {
     /** */
     private final AtomicLong diagFutId = new AtomicLong();
 
+    /** */
+    private final Map<UUID, byte[]> allNodesMetrics = new ConcurrentHashMap<>();
+
+    /** */
+    private final JdkMarshaller marsh = new JdkMarshaller();
+
+    /** */
+    private DiscoveryMetricsProvider metricsProvider;
+
+    /** */
+    private boolean sndMetrics;
+
     /**
      * @param ctx Kernal context.
      */
@@ -111,6 +134,8 @@ public class ClusterProcessor extends GridProcessorAdapter {
         notifyEnabled.set(IgniteSystemProperties.getBoolean(IGNITE_UPDATE_NOTIFIER, true));
 
         cluster = new IgniteClusterImpl(ctx);
+
+        sndMetrics = !(ctx.config().getDiscoverySpi() instanceof TcpDiscoverySpi);
     }
 
     /**
@@ -120,33 +145,31 @@ public class ClusterProcessor extends GridProcessorAdapter {
         return getBoolean(IGNITE_DIAGNOSTIC_ENABLED, true);
     }
 
-    /** */
-    private final JdkMarshaller marsh = new JdkMarshaller();
-
     /**
      * @throws IgniteCheckedException If failed.
      */
     public void initDiagnosticListeners() throws IgniteCheckedException {
         ctx.event().addLocalEventListener(new GridLocalEventListener() {
-                @Override public void onEvent(Event evt) {
-                    assert evt instanceof DiscoveryEvent;
-                    assert evt.type() == EVT_NODE_FAILED || evt.type() == EVT_NODE_LEFT;
+            @Override public void onEvent(Event evt) {
+                assert evt instanceof DiscoveryEvent;
+                assert evt.type() == EVT_NODE_FAILED || evt.type() == EVT_NODE_LEFT;
 
-                    DiscoveryEvent discoEvt = (DiscoveryEvent)evt;
+                DiscoveryEvent discoEvt = (DiscoveryEvent)evt;
 
-                    UUID nodeId = discoEvt.eventNode().id();
+                UUID nodeId = discoEvt.eventNode().id();
 
-                    ConcurrentHashMap<Long, InternalDiagnosticFuture> futs = diagnosticFutMap.get();
+                ConcurrentHashMap<Long, InternalDiagnosticFuture> futs = diagnosticFutMap.get();
 
-                    if (futs != null) {
-                        for (InternalDiagnosticFuture fut : futs.values()) {
-                            if (fut.nodeId.equals(nodeId))
-                                fut.onDone(new IgniteDiagnosticInfo("Target node failed: " + nodeId));
-                        }
+                if (futs != null) {
+                    for (InternalDiagnosticFuture fut : futs.values()) {
+                        if (fut.nodeId.equals(nodeId))
+                            fut.onDone(new IgniteDiagnosticInfo("Target node failed: " + nodeId));
                     }
                 }
-            },
-            EVT_NODE_FAILED, EVT_NODE_LEFT);
+
+                allNodesMetrics.remove(nodeId);
+            }
+        }, EVT_NODE_FAILED, EVT_NODE_LEFT);
 
         ctx.io().addMessageListener(TOPIC_INTERNAL_DIAGNOSTIC, new GridMessageListener() {
             @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
@@ -233,6 +256,17 @@ public class ClusterProcessor extends GridProcessorAdapter {
                     U.warn(diagnosticLog, "Received unexpected message: " + msg);
             }
         });
+
+        if (sndMetrics) {
+            ctx.io().addMessageListener(TOPIC_METRICS, new GridMessageListener() {
+                @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
+                    if (msg instanceof ClusterMetricsUpdateMessage)
+                        processMetricsUpdateMessage(nodeId, (ClusterMetricsUpdateMessage)msg);
+                    else
+                        U.warn(log, "Received unexpected message for TOPIC_METRICS: " + msg);
+                }
+            });
+        }
     }
 
     /**
@@ -296,7 +330,6 @@ public class ClusterProcessor extends GridProcessorAdapter {
         }
     }
 
-
     /**
      * @param vals collection to seek through.
      */
@@ -334,6 +367,14 @@ public class ClusterProcessor extends GridProcessorAdapter {
                     log.debug("Failed to create GridUpdateNotifier: " + e);
             }
         }
+
+        if (sndMetrics) {
+            metricsProvider = ctx.discovery().createMetricsProvider();
+
+            long updateFreq = ctx.config().getMetricsUpdateFrequency();
+
+            ctx.timeout().addTimeoutObject(new MetricsUpdateTimeoutObject(updateFreq));
+        }
     }
 
     /** {@inheritDoc} */
@@ -352,6 +393,133 @@ public class ClusterProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param sndNodeId Sender node ID.
+     * @param msg Message.
+     */
+    private void processMetricsUpdateMessage(UUID sndNodeId, ClusterMetricsUpdateMessage msg) {
+        byte[] nodeMetrics = msg.nodeMetrics();
+
+        if (nodeMetrics != null) {
+            assert msg.allNodesMetrics() == null;
+
+            allNodesMetrics.put(sndNodeId, nodeMetrics);
+
+            updateNodeMetrics(ctx.discovery().discoCache(), sndNodeId, nodeMetrics);
+        }
+        else {
+            Map<UUID, byte[]> allNodesMetrics = msg.allNodesMetrics();
+
+            assert allNodesMetrics != null;
+
+            DiscoCache discoCache = ctx.discovery().discoCache();
+
+            for (Map.Entry<UUID, byte[]> e : allNodesMetrics.entrySet()) {
+                if (!ctx.localNodeId().equals(e.getKey()))
+                    updateNodeMetrics(discoCache, e.getKey(), e.getValue());
+            }
+        }
+    }
+
+    /**
+     * @param discoCache Discovery data cache.
+     * @param nodeId Node ID.
+     * @param metricsBytes Marshalled metrics.
+     */
+    private void updateNodeMetrics(DiscoCache discoCache, UUID nodeId, byte[] metricsBytes) {
+        ClusterNode node = discoCache.node(nodeId);
+
+        if (node == null || !discoCache.alive(nodeId))
+            return;
+
+        try {
+            ClusterNodeMetrics metrics = U.unmarshalZip(ctx.config().getMarshaller(), metricsBytes, null);
+
+            assert node instanceof IgniteClusterNode : node;
+
+            IgniteClusterNode node0 = (IgniteClusterNode)node;
+
+            node0.setMetrics(ClusterMetricsSnapshot.deserialize(metrics.metrics(), 0));
+            node0.setCacheMetrics(metrics.cacheMetrics());
+
+            ctx.discovery().metricsUpdateEvent(discoCache, node0);
+        }
+        catch (IgniteCheckedException e) {
+            U.warn(log, "Failed to unmarshal node metrics: " + e);
+        }
+    }
+
+    /**
+     *
+     */
+    private void updateMetrics() {
+        if (ctx.isStopping() || ctx.clientDisconnected())
+            return;
+
+        ClusterNode oldest = ctx.discovery().oldestAliveServerNode(AffinityTopologyVersion.NONE);
+
+        if (oldest == null)
+            return;
+
+        if (ctx.localNodeId().equals(oldest.id())) {
+            IgniteClusterNode locNode = (IgniteClusterNode)ctx.discovery().localNode();
+
+            locNode.setMetrics(metricsProvider.metrics());
+            locNode.setCacheMetrics(metricsProvider.cacheMetrics());
+
+            ClusterNodeMetrics metrics = new ClusterNodeMetrics(locNode.metrics(), locNode.cacheMetrics());
+
+            try {
+                byte[] metricsBytes = U.zip(U.marshal(ctx.config().getMarshaller(), metrics));
+
+                allNodesMetrics.put(ctx.localNodeId(), metricsBytes);
+            }
+            catch (IgniteCheckedException e) {
+                U.warn(log, "Failed to marshal local node metrics: " + e, e);
+            }
+
+            ctx.discovery().metricsUpdateEvent(ctx.discovery().discoCache(), locNode);
+
+            Collection<ClusterNode> allNodes = ctx.discovery().allNodes();
+
+            ClusterMetricsUpdateMessage msg = new ClusterMetricsUpdateMessage(new HashMap<>(allNodesMetrics));
+
+            for (ClusterNode node : allNodes) {
+                if (ctx.localNodeId().equals(node.id()) || !ctx.discovery().alive(node.id()))
+                    continue;
+
+                try {
+                    ctx.io().sendToGridTopic(node, TOPIC_METRICS, msg, GridIoPolicy.SYSTEM_POOL);
+                }
+                catch (ClusterTopologyCheckedException e) {
+                    if (log.isDebugEnabled())
+                        log.debug("Failed to send metrics update, node failed: " + e);
+                }
+                catch (IgniteCheckedException e) {
+                    U.warn(log, "Failed to send metrics update: " + e, e);
+                }
+            }
+        }
+        else {
+            ClusterNodeMetrics metrics = new ClusterNodeMetrics(metricsProvider.metrics(), metricsProvider.cacheMetrics());
+
+            try {
+                byte[] metricsBytes = U.zip(U.marshal(ctx.config().getMarshaller(), metrics));
+
+                ClusterMetricsUpdateMessage msg = new ClusterMetricsUpdateMessage(metricsBytes);
+
+                ctx.io().sendToGridTopic(oldest, TOPIC_METRICS, msg, GridIoPolicy.SYSTEM_POOL);
+            }
+            catch (ClusterTopologyCheckedException e) {
+                if (log.isDebugEnabled())
+                    log.debug("Failed to send metrics update to oldest, node failed: " + e);
+            }
+            catch (IgniteCheckedException e) {
+                LT.warn(log, e, "Failed to send metrics update to oldest: " + e, false, false);
+            }
+        }
+    }
+
+    /**
      * Disables update notifier.
      */
     public void disableUpdateNotifier() {
@@ -571,4 +739,51 @@ public class ClusterProcessor extends GridProcessorAdapter {
             return S.toString(InternalDiagnosticFuture.class, this);
         }
     }
+
+    /**
+     *
+     */
+    private class MetricsUpdateTimeoutObject implements GridTimeoutObject, Runnable {
+        /** */
+        private final IgniteUuid id = IgniteUuid.randomUuid();
+
+        /** */
+        private long endTime;
+
+        /** */
+        private final long timeout;
+
+        /**
+         * @param timeout Timeout.
+         */
+        MetricsUpdateTimeoutObject(long timeout) {
+            this.timeout = timeout;
+
+            endTime = U.currentTimeMillis() + timeout;
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgniteUuid timeoutId() {
+            return id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public long endTime() {
+            return endTime;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            updateMetrics();
+
+            endTime = U.currentTimeMillis() + timeout;
+
+            ctx.timeout().addTimeoutObject(this);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onTimeout() {
+            ctx.getSystemExecutorService().execute(this);
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java
index e9754d1..928c619 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java
@@ -63,6 +63,11 @@ public abstract class AbstractContinuousMessage implements DiscoveryCustomMessag
     }
 
     /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
         AffinityTopologyVersion topVer, DiscoCache discoCache) {
         throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineInfo.java
new file mode 100644
index 0000000..fc0f181
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineInfo.java
@@ -0,0 +1,100 @@
+/*
+ * 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.continuous;
+
+import java.io.Serializable;
+import java.util.UUID;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+class ContinuousRoutineInfo implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    UUID srcNodeId;
+
+    /** */
+    final UUID routineId;
+
+    /** */
+    final byte[] hnd;
+
+    /** */
+    final byte[] nodeFilter;
+
+    /** */
+    final int bufSize;
+
+    /** */
+    final long interval;
+
+    /** */
+    final boolean autoUnsubscribe;
+
+    /** */
+    transient boolean disconnected;
+
+    /**
+     * @param srcNodeId Source node ID.
+     * @param routineId Routine ID.
+     * @param hnd Marshalled handler.
+     * @param nodeFilter Marshalled node filter.
+     * @param bufSize Handler buffer size.
+     * @param interval Time interval.
+     * @param autoUnsubscribe Auto unsubscribe flag.
+     */
+    ContinuousRoutineInfo(
+        UUID srcNodeId,
+        UUID routineId,
+        byte[] hnd,
+        byte[] nodeFilter,
+        int bufSize,
+        long interval,
+        boolean autoUnsubscribe)
+    {
+        this.srcNodeId = srcNodeId;
+        this.routineId = routineId;
+        this.hnd = hnd;
+        this.nodeFilter = nodeFilter;
+        this.bufSize = bufSize;
+        this.interval = interval;
+        this.autoUnsubscribe = autoUnsubscribe;
+    }
+
+    /**
+     * @param srcNodeId Source node ID.
+     */
+    void sourceNodeId(UUID srcNodeId) {
+        this.srcNodeId = srcNodeId;
+    }
+
+    /**
+     *
+     */
+    void onDisconnected() {
+        disconnected = true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(ContinuousRoutineInfo.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineStartResultMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineStartResultMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineStartResultMessage.java
new file mode 100644
index 0000000..581ac60
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineStartResultMessage.java
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.continuous;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionPartialCountersMap;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+public class ContinuousRoutineStartResultMessage implements Message {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private static final int ERROR_FLAG = 0x01;
+
+    /** */
+    private UUID routineId;
+
+    /** */
+    private byte[] errBytes;
+
+    /** */
+    private byte[] cntrsMapBytes;
+
+    /** */
+    private int flags;
+
+    /**
+     *
+     */
+    public ContinuousRoutineStartResultMessage() {
+        // No-op.
+    }
+
+    /**
+     * @param routineId Routine ID.
+     * @param cntrsMapBytes Marshalled {@link CachePartitionPartialCountersMap}.
+     * @param errBytes Error bytes.
+     * @param err {@code True} if failed to start routine.
+     */
+    ContinuousRoutineStartResultMessage(UUID routineId, byte[] cntrsMapBytes, byte[] errBytes, boolean err) {
+        this.routineId = routineId;
+        this.cntrsMapBytes = cntrsMapBytes;
+        this.errBytes = errBytes;
+
+        if (err)
+            flags |= ERROR_FLAG;
+    }
+
+    /**
+     * @return Marshalled {@link CachePartitionPartialCountersMap}.
+     */
+    @Nullable byte[] countersMapBytes() {
+        return cntrsMapBytes;
+    }
+
+    /**
+     * @return {@code True} if failed to start routine.
+     */
+    boolean error() {
+        return (flags & ERROR_FLAG) != 0;
+    }
+
+    /**
+     * @return Routine ID.
+     */
+    UUID routineId() {
+        return routineId;
+    }
+
+    /**
+     * @return Error bytes.
+     */
+    @Nullable byte[] errorBytes() {
+        return errBytes;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeByteArray("cntrsMapBytes", cntrsMapBytes))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeByteArray("errBytes", errBytes))
+                    return false;
+
+                writer.incrementState();
+
+            case 2:
+                if (!writer.writeInt("flags", flags))
+                    return false;
+
+                writer.incrementState();
+
+            case 3:
+                if (!writer.writeUuid("routineId", routineId))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                cntrsMapBytes = reader.readByteArray("cntrsMapBytes");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                errBytes = reader.readByteArray("errBytes");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 2:
+                flags = reader.readInt("flags");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 3:
+                routineId = reader.readUuid("routineId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(ContinuousRoutineStartResultMessage.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public short directType() {
+        return 134;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 4;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(ContinuousRoutineStartResultMessage.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesCommonDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesCommonDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesCommonDiscoveryData.java
new file mode 100644
index 0000000..d29de89
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesCommonDiscoveryData.java
@@ -0,0 +1,45 @@
+/*
+ * 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.continuous;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+public class ContinuousRoutinesCommonDiscoveryData implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    final List<ContinuousRoutineInfo> startedRoutines;
+
+    /**
+     * @param startedRoutines Routines started in cluster.
+     */
+    ContinuousRoutinesCommonDiscoveryData(List<ContinuousRoutineInfo> startedRoutines) {
+        this.startedRoutines = startedRoutines;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(ContinuousRoutinesCommonDiscoveryData.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesInfo.java
new file mode 100644
index 0000000..ad24ff1
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesInfo.java
@@ -0,0 +1,132 @@
+/*
+ * 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.continuous;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+
+import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.CONTINUOUS_PROC;
+
+/**
+ *
+ */
+class ContinuousRoutinesInfo {
+    /** */
+    private final Map<UUID, ContinuousRoutineInfo> startedRoutines = new HashMap<>();
+
+    /**
+     * @param dataBag Discovery data bag.
+     */
+    void collectGridNodeData(DiscoveryDataBag dataBag) {
+        synchronized (startedRoutines) {
+            if (!dataBag.commonDataCollectedFor(CONTINUOUS_PROC.ordinal()))
+                dataBag.addGridCommonData(CONTINUOUS_PROC.ordinal(),
+                    new ContinuousRoutinesCommonDiscoveryData(new ArrayList<>(startedRoutines.values())));
+        }
+    }
+
+    /**
+     * @param dataBag Discovery data bag.
+     */
+    void collectJoiningNodeData(DiscoveryDataBag dataBag) {
+        synchronized (startedRoutines) {
+            for (ContinuousRoutineInfo info : startedRoutines.values()) {
+                if (info.disconnected)
+                    info.sourceNodeId(dataBag.joiningNodeId());
+            }
+
+            dataBag.addJoiningNodeData(CONTINUOUS_PROC.ordinal(),
+                new ContinuousRoutinesJoiningNodeDiscoveryData(new ArrayList<>(startedRoutines.values())));
+        }
+    }
+
+    /**
+     * @param info Routine info.
+     */
+    void addRoutineInfo(ContinuousRoutineInfo info) {
+        synchronized (startedRoutines) {
+            startedRoutines.put(info.routineId, info);
+        }
+    }
+
+    /**
+     * @param routineId Routine ID.
+     * @return {@code True} if routine exists.
+     */
+    boolean routineExists(UUID routineId) {
+        synchronized (startedRoutines) {
+            return startedRoutines.containsKey(routineId);
+        }
+    }
+
+    /**
+     * @param routineId Routine ID.
+     */
+    void removeRoutine(UUID routineId) {
+        synchronized (startedRoutines) {
+            startedRoutines.remove(routineId);
+        }
+    }
+
+    /**
+     * @param locRoutines Routines IDs which can survive reconnect.
+     */
+    void onClientDisconnected(Collection<UUID> locRoutines) {
+        synchronized (startedRoutines) {
+            for (Iterator<Map.Entry<UUID, ContinuousRoutineInfo>> it = startedRoutines.entrySet().iterator(); it.hasNext();) {
+                Map.Entry<UUID, ContinuousRoutineInfo> e = it.next();
+
+                ContinuousRoutineInfo info = e.getValue();
+
+                if (!locRoutines.contains(info.routineId))
+                    it.remove();
+                else
+                    info.onDisconnected();
+            }
+        }
+    }
+
+    /**
+     * Removes all routines with autoUnsubscribe=false started by given node.
+     *
+     * @param nodeId Node ID.
+     */
+    void onNodeFail(UUID nodeId) {
+        synchronized (startedRoutines) {
+            for (Iterator<Map.Entry<UUID, ContinuousRoutineInfo>> it = startedRoutines.entrySet().iterator(); it.hasNext();) {
+                Map.Entry<UUID, ContinuousRoutineInfo> e = it.next();
+
+                ContinuousRoutineInfo info = e.getValue();
+
+                if (info.autoUnsubscribe && info.srcNodeId.equals(nodeId))
+                    it.remove();
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(ContinuousRoutinesInfo.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesJoiningNodeDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesJoiningNodeDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesJoiningNodeDiscoveryData.java
new file mode 100644
index 0000000..9be6ef8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesJoiningNodeDiscoveryData.java
@@ -0,0 +1,45 @@
+/*
+ * 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.continuous;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+public class ContinuousRoutinesJoiningNodeDiscoveryData implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    final List<ContinuousRoutineInfo> startedRoutines;
+
+    /**
+     * @param startedRoutines Routines registered on nodes, to be started in cluster.
+     */
+    ContinuousRoutinesJoiningNodeDiscoveryData(List<ContinuousRoutineInfo> startedRoutines) {
+        this.startedRoutines = startedRoutines;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(ContinuousRoutinesJoiningNodeDiscoveryData.class, this);
+    }
+}


[04/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java
new file mode 100644
index 0000000..7708358
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java
@@ -0,0 +1,4464 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.ByteArrayInputStream;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.DataFormatException;
+import java.util.zip.Deflater;
+import java.util.zip.Inflater;
+import java.util.zip.InflaterInputStream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteClientDisconnectedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteInterruptedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CommunicationFailureResolver;
+import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.ClusterMetricsSnapshot;
+import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
+import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.IgnitionEx;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.events.DiscoveryCustomEvent;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpiInternalListener;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.security.SecurityContext;
+import org.apache.ignite.internal.util.GridLongList;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.io.GridByteArrayOutputStream;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.LT;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.marshaller.MarshallerUtils;
+import org.apache.ignite.marshaller.jdk.JdkMarshaller;
+import org.apache.ignite.plugin.security.SecurityCredentials;
+import org.apache.ignite.spi.IgniteNodeValidationResult;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.IgniteSpiTimeoutObject;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.apache.ignite.spi.discovery.DiscoverySpiDataExchange;
+import org.apache.ignite.spi.discovery.DiscoverySpiListener;
+import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator;
+import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.data.Stat;
+import org.jboss.netty.util.internal.ConcurrentHashMap;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED;
+import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_RECONNECTED;
+import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
+import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
+import static org.apache.ignite.events.EventType.EVT_NODE_SEGMENTED;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGNITE_INSTANCE_NAME;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2;
+import static org.apache.zookeeper.CreateMode.EPHEMERAL_SEQUENTIAL;
+import static org.apache.zookeeper.CreateMode.PERSISTENT;
+
+/**
+ * Zookeeper Discovery Impl.
+ */
+public class ZookeeperDiscoveryImpl {
+    /** */
+    static final String IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD = "IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD";
+
+    /** */
+    static final String IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_TIMEOUT = "IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_TIMEOUT";
+
+    /** */
+    static final String IGNITE_ZOOKEEPER_DISCOVERY_SPI_MAX_EVTS = "IGNITE_ZOOKEEPER_DISCOVERY_SPI_MAX_EVTS";
+
+    /** */
+    private static final String IGNITE_ZOOKEEPER_DISCOVERY_SPI_EVTS_THROTTLE = "IGNITE_ZOOKEEPER_DISCOVERY_SPI_EVTS_THROTTLE";
+
+    /** */
+    final ZookeeperDiscoverySpi spi;
+
+    /** */
+    private final String igniteInstanceName;
+
+    /** */
+    private final String connectString;
+
+    /** */
+    private final int sesTimeout;
+
+    /** */
+    private final JdkMarshaller marsh = new JdkMarshaller();
+
+    /** */
+    private final ZkIgnitePaths zkPaths;
+
+    /** */
+    private final IgniteLogger log;
+
+    /** */
+    final GridSpinBusyLock busyLock = new GridSpinBusyLock();
+
+    /** */
+    private final ZookeeperClusterNode locNode;
+
+    /** */
+    private final DiscoverySpiListener lsnr;
+
+    /** */
+    private final DiscoverySpiDataExchange exchange;
+
+    /** */
+    private final boolean clientReconnectEnabled;
+
+    /** */
+    private final GridFutureAdapter<Void> joinFut = new GridFutureAdapter<>();
+
+    /** */
+    private final int evtsAckThreshold;
+
+    /** */
+    private IgniteThreadPoolExecutor utilityPool;
+
+    /** */
+    private ZkRuntimeState rtState;
+
+    /** */
+    private volatile ConnectionState connState = ConnectionState.STARTED;
+
+    /** */
+    private final AtomicBoolean stop = new AtomicBoolean();
+
+    /** */
+    private final Object stateMux = new Object();
+
+    /** */
+    public volatile IgniteDiscoverySpiInternalListener internalLsnr;
+
+    /** */
+    private final ConcurrentHashMap<Long, PingFuture> pingFuts = new ConcurrentHashMap<>();
+
+    /** */
+    private final AtomicReference<ZkCommunicationErrorProcessFuture> commErrProcFut = new AtomicReference<>();
+
+    /** */
+    private long prevSavedEvtsTopVer;
+
+    /**
+     * @param spi Discovery SPI.
+     * @param igniteInstanceName Instance name.
+     * @param log Logger.
+     * @param zkRootPath Zookeeper base path node all nodes.
+     * @param locNode Local node instance.
+     * @param lsnr Discovery events listener.
+     * @param exchange Discovery data exchange.
+     * @param internalLsnr Internal listener (used for testing only).
+     */
+    public ZookeeperDiscoveryImpl(
+        ZookeeperDiscoverySpi spi,
+        String igniteInstanceName,
+        IgniteLogger log,
+        String zkRootPath,
+        ZookeeperClusterNode locNode,
+        DiscoverySpiListener lsnr,
+        DiscoverySpiDataExchange exchange,
+        IgniteDiscoverySpiInternalListener internalLsnr) {
+        assert locNode.id() != null && locNode.isLocal() : locNode;
+
+        MarshallerUtils.setNodeName(marsh, igniteInstanceName);
+
+        zkPaths = new ZkIgnitePaths(zkRootPath);
+
+        this.spi = spi;
+        this.igniteInstanceName = igniteInstanceName;
+        this.connectString = spi.getZkConnectionString();
+        this.sesTimeout = (int)spi.getSessionTimeout();
+        this.log = log.getLogger(getClass());
+        this.locNode = locNode;
+        this.lsnr = lsnr;
+        this.exchange = exchange;
+        this.clientReconnectEnabled = locNode.isClient() && !spi.isClientReconnectDisabled();
+
+        int evtsAckThreshold = IgniteSystemProperties.getInteger(IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD, 5);
+
+        if (evtsAckThreshold <= 0)
+            evtsAckThreshold = 1;
+
+        this.evtsAckThreshold = evtsAckThreshold;
+
+        if (internalLsnr != null)
+            this.internalLsnr = internalLsnr;
+    }
+
+    /**
+     * @return Exception.
+     */
+    private static IgniteClientDisconnectedCheckedException disconnectError() {
+        return new IgniteClientDisconnectedCheckedException(null, "Client node disconnected.");
+    }
+
+    /**
+     * @return Logger.
+     */
+    IgniteLogger log() {
+        return log;
+    }
+
+    /**
+     * @return Local node instance.
+     */
+    public ClusterNode localNode() {
+        return locNode;
+    }
+
+    /**
+     * @param nodeId Node ID.
+     * @return Node instance.
+     */
+    @Nullable public ZookeeperClusterNode node(UUID nodeId) {
+        assert nodeId != null;
+
+        return rtState.top.nodesById.get(nodeId);
+    }
+
+    /**
+     * @param nodeOrder Node order.
+     * @return Node instance.
+     */
+    @Nullable public ZookeeperClusterNode node(long nodeOrder) {
+        assert nodeOrder > 0 : nodeOrder;
+
+        return rtState.top.nodesByOrder.get(nodeOrder);
+    }
+
+    /**
+     * @param fut Future to remove.
+     */
+    void clearCommunicationErrorProcessFuture(ZkCommunicationErrorProcessFuture fut) {
+        assert fut.isDone() : fut;
+
+        commErrProcFut.compareAndSet(fut, null);
+    }
+
+    /**
+     * @param node0 Problem node ID
+     * @param err Connect error.
+     */
+    public void resolveCommunicationError(ClusterNode node0, Exception err) {
+        ZookeeperClusterNode node = node(node0.id());
+
+        if (node == null)
+            throw new IgniteSpiException(new ClusterTopologyCheckedException("Node failed: " + node0.id()));
+
+        IgniteInternalFuture<Boolean> nodeStatusFut;
+
+        for (;;) {
+            checkState();
+
+            ZkCommunicationErrorProcessFuture fut = commErrProcFut.get();
+
+            if (fut == null || fut.isDone()) {
+                ZkCommunicationErrorProcessFuture newFut = ZkCommunicationErrorProcessFuture.createOnCommunicationError(
+                    this,
+                    node.sessionTimeout() + 1000);
+
+                if (commErrProcFut.compareAndSet(fut, newFut)) {
+                    fut = newFut;
+
+                    if (log.isInfoEnabled()) {
+                        log.info("Created new communication error process future [errNode=" + node0.id() +
+                            ", err=" + err + ']');
+                    }
+
+                    try {
+                        checkState();
+                    }
+                    catch (Exception e) {
+                        fut.onError(e);
+
+                        throw e;
+                    }
+
+                    fut.scheduleCheckOnTimeout();
+                }
+                else {
+                    fut = commErrProcFut.get();
+
+                    if (fut == null)
+                        continue;
+                }
+            }
+
+            nodeStatusFut = fut.nodeStatusFuture(node);
+
+            if (nodeStatusFut != null)
+                break;
+            else {
+                try {
+                    fut.get();
+                }
+                catch (IgniteCheckedException e) {
+                    U.warn(log, "Previous communication error process future failed: " + e);
+                }
+            }
+        }
+
+        try {
+            if (!nodeStatusFut.get())
+                throw new IgniteSpiException(new ClusterTopologyCheckedException("Node failed: " + node0.id()));
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteSpiException(e);
+        }
+    }
+
+    /**
+     * @param nodeId Node ID.
+     * @return Ping result.
+     */
+    public boolean pingNode(UUID nodeId) {
+        checkState();
+
+        ZkRuntimeState rtState = this.rtState;
+
+        ZookeeperClusterNode node = rtState.top.nodesById.get(nodeId);
+
+        if (node == null)
+            return false;
+
+        if (node.isLocal())
+            return true;
+
+        PingFuture fut = pingFuts.get(node.order());
+
+        if (fut == null) {
+            fut = new PingFuture(rtState, node);
+
+            PingFuture old = pingFuts.putIfAbsent(node.order(), fut);
+
+            if (old == null) {
+                if (fut.checkNodeAndState())
+                    spi.getSpiContext().addTimeoutObject(fut);
+                else
+                    assert fut.isDone();
+            }
+            else
+                fut = old;
+        }
+
+        try {
+            return fut.get();
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteSpiException(e);
+        }
+    }
+
+    /**
+     * @param nodeId Node ID.
+     * @param warning Warning.
+     */
+    public void failNode(UUID nodeId, @Nullable String warning) {
+        ZookeeperClusterNode node = rtState.top.nodesById.get(nodeId);
+
+        if (node == null) {
+            if (log.isDebugEnabled())
+                log.debug("Ignore forcible node fail request, node does not exist: " + nodeId);
+
+            return;
+        }
+
+        if (!node.isClient()) {
+            U.warn(log, "Ignore forcible node fail request for non-client node: " + node);
+
+            return;
+        }
+
+        sendCustomMessage(new ZkForceNodeFailMessage(node.internalId(), warning));
+    }
+
+    /**
+     *
+     */
+    public void reconnect() {
+        assert clientReconnectEnabled;
+
+        synchronized (stateMux) {
+            if (connState == ConnectionState.STARTED) {
+                connState = ConnectionState.DISCONNECTED;
+
+                rtState.onCloseStart(disconnectError());
+            }
+            else
+                return;
+        }
+
+        busyLock.block();
+
+        busyLock.unblock();
+
+        rtState.zkClient.close();
+
+        UUID newId = UUID.randomUUID();
+
+        U.quietAndWarn(log, "Local node will try to reconnect to cluster with new id due to network problems [" +
+            "newId=" + newId +
+            ", prevId=" + locNode.id() +
+            ", locNode=" + locNode + ']');
+
+        runInWorkerThread(new ReconnectClosure(newId));
+    }
+
+    /**
+     * @param newId New ID.
+     */
+    private void doReconnect(UUID newId) {
+        if (rtState.joined) {
+            assert rtState.evtsData != null;
+
+            lsnr.onDiscovery(EVT_CLIENT_NODE_DISCONNECTED,
+                rtState.evtsData.topVer,
+                locNode,
+                rtState.top.topologySnapshot(),
+                Collections.<Long, Collection<ClusterNode>>emptyMap(),
+                null);
+        }
+
+        try {
+            locNode.onClientDisconnected(newId);
+
+            joinTopology(rtState);
+        }
+        catch (Exception e) {
+            if (stopping()) {
+                if (log.isDebugEnabled())
+                    log.debug("Reconnect failed, node is stopping [err=" + e + ']');
+
+                return;
+            }
+
+            U.error(log, "Failed to reconnect: " + e, e);
+
+            onSegmented(e);
+        }
+    }
+
+    /**
+     * @return {@code True} if started to stop.
+     */
+    private boolean stopping() {
+        if (stop.get())
+            return true;
+
+        synchronized (stateMux) {
+            if (connState == ConnectionState.STOPPED)
+                return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * @param e Error.
+     */
+    private void onSegmented(Exception e) {
+        rtState.errForClose = e;
+
+        if (rtState.joined || joinFut.isDone()) {
+            synchronized (stateMux) {
+                connState = ConnectionState.STOPPED;
+            }
+
+            notifySegmented();
+        }
+        else
+            joinFut.onDone(e);
+    }
+
+    /**
+     *
+     */
+    private void notifySegmented() {
+        List<ClusterNode> nodes = rtState.top.topologySnapshot();
+
+        if (nodes.isEmpty())
+            nodes = Collections.singletonList((ClusterNode)locNode);
+
+        lsnr.onDiscovery(EVT_NODE_SEGMENTED,
+            rtState.evtsData != null ? rtState.evtsData.topVer : 1L,
+            locNode,
+            nodes,
+            Collections.<Long, Collection<ClusterNode>>emptyMap(),
+            null);
+    }
+
+    /**
+     * @return Remote nodes.
+     */
+    public Collection<ClusterNode> remoteNodes() {
+        checkState();
+
+        return rtState.top.remoteNodes();
+    }
+
+    /**
+     *
+     */
+    private void checkState() {
+        switch (connState) {
+            case STARTED:
+                break;
+
+            case STOPPED:
+                throw new IgniteSpiException("Node stopped.");
+
+            case DISCONNECTED:
+                throw new IgniteClientDisconnectedException(null, "Client is disconnected.");
+        }
+    }
+
+    /**
+     * @param nodeId Node ID.
+     * @return {@code True} if node joined or joining topology.
+     */
+    public boolean knownNode(UUID nodeId) {
+        while (!busyLock.enterBusy())
+            checkState();
+
+        try {
+            List<String> children = rtState.zkClient.getChildren(zkPaths.aliveNodesDir);
+
+            for (int i = 0; i < children.size(); i++) {
+                UUID id = ZkIgnitePaths.aliveNodeId(children.get(i));
+
+                if (nodeId.equals(id))
+                    return true;
+            }
+
+            return false;
+        }
+        catch (ZookeeperClientFailedException e) {
+            if (clientReconnectEnabled)
+                throw new IgniteClientDisconnectedException(null, "Client is disconnected.");
+
+            throw new IgniteException(e);
+        }
+        catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IgniteInterruptedException(e);
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * @param msg Message.
+     */
+    public void sendCustomMessage(DiscoverySpiCustomMessage msg) {
+        assert msg != null;
+
+        byte[] msgBytes;
+
+        try {
+            msgBytes = marshalZip(msg);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteSpiException("Failed to marshal custom message: " + msg, e);
+        }
+
+        while (!busyLock.enterBusy())
+            checkState();
+
+        try {
+            ZookeeperClient zkClient = rtState.zkClient;
+
+            saveCustomMessage(zkClient, msgBytes);
+        }
+        catch (ZookeeperClientFailedException e) {
+            if (clientReconnectEnabled)
+                throw new IgniteClientDisconnectedException(null, "Client is disconnected.");
+
+            throw new IgniteException(e);
+        }
+        catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IgniteInterruptedException(e);
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * @param zkClient Client.
+     * @param msgBytes Marshalled message.
+     * @throws ZookeeperClientFailedException If connection to zk was lost.
+     * @throws InterruptedException If interrupted.
+     */
+    private void saveCustomMessage(ZookeeperClient zkClient, byte[] msgBytes)
+        throws ZookeeperClientFailedException, InterruptedException
+    {
+        String prefix = UUID.randomUUID().toString();
+
+        int partCnt = 1;
+
+        int overhead = 10;
+
+        UUID locId = locNode.id();
+
+        String path = zkPaths.createCustomEventPath(prefix, locId, partCnt);
+
+        if (zkClient.needSplitNodeData(path, msgBytes, overhead)) {
+            List<byte[]> parts = zkClient.splitNodeData(path, msgBytes, overhead);
+
+            String partsBasePath = zkPaths.customEventPartsBasePath(prefix, locId);
+
+            saveMultipleParts(zkClient, partsBasePath, parts);
+
+            msgBytes = null;
+
+            partCnt = parts.size();
+        }
+
+        zkClient.createSequential(prefix,
+            zkPaths.customEvtsDir,
+            zkPaths.createCustomEventPath(prefix, locId, partCnt),
+            msgBytes,
+            CreateMode.PERSISTENT_SEQUENTIAL);
+    }
+
+    /**
+     * @return Cluster start time.
+     */
+    public long gridStartTime() {
+        return rtState.gridStartTime;
+    }
+
+    /**
+     * Starts join procedure and waits for {@link EventType#EVT_NODE_JOINED} event for local node.
+     *
+     * @throws InterruptedException If interrupted.
+     */
+    public void startJoinAndWait() throws InterruptedException {
+        joinTopology(null);
+
+        for (;;) {
+            try {
+                joinFut.get(10_000);
+
+                break;
+            }
+            catch (IgniteFutureTimeoutCheckedException e) {
+                U.warn(log, "Waiting for local join event [nodeId=" + locNode.id() + ", name=" + igniteInstanceName + ']');
+            }
+            catch (Exception e) {
+                IgniteSpiException spiErr = X.cause(e, IgniteSpiException.class);
+
+                if (spiErr != null)
+                    throw spiErr;
+
+                throw new IgniteSpiException("Failed to join cluster", e);
+            }
+        }
+    }
+
+    /**
+     * @param prevState Previous state in case of connect retry.
+     * @throws InterruptedException If interrupted.
+     */
+    private void joinTopology(@Nullable ZkRuntimeState prevState) throws InterruptedException {
+        if (!busyLock.enterBusy())
+            return;
+
+        try {
+            boolean reconnect = prevState != null;
+
+            // Need fire EVT_CLIENT_NODE_RECONNECTED event if reconnect after already joined.
+            boolean prevJoined = prevState != null && prevState.joined;
+
+            IgniteDiscoverySpiInternalListener internalLsnr = this.internalLsnr;
+
+            if (internalLsnr != null)
+                internalLsnr.beforeJoin(locNode, log);
+
+            if (locNode.isClient() && reconnect)
+                locNode.setAttributes(spi.getSpiContext().nodeAttributes());
+
+            marshalCredentialsOnJoin(locNode);
+
+            synchronized (stateMux) {
+                if (connState == ConnectionState.STOPPED)
+                    return;
+
+                connState = ConnectionState.STARTED;
+            }
+
+            ZkRuntimeState rtState = this.rtState = new ZkRuntimeState(prevJoined);
+
+            DiscoveryDataBag discoDataBag = new DiscoveryDataBag(locNode.id(), locNode.isClient());
+
+            exchange.collect(discoDataBag);
+
+            ZkJoiningNodeData joinData = new ZkJoiningNodeData(locNode, discoDataBag.joiningNodeData());
+
+            byte[] joinDataBytes;
+
+            try {
+                joinDataBytes = marshalZip(joinData);
+            }
+            catch (Exception e) {
+                throw new IgniteSpiException("Failed to marshal joining node data", e);
+            }
+
+            try {
+                rtState.zkClient = new ZookeeperClient(
+                    igniteInstanceName,
+                    log,
+                    connectString,
+                    sesTimeout,
+                    new ConnectionLossListener());
+            }
+            catch (Exception e) {
+                throw new IgniteSpiException("Failed to create Zookeeper client", e);
+            }
+
+            startJoin(rtState, prevState, joinDataBytes);
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * @throws InterruptedException If interrupted.
+     */
+    private void initZkNodes() throws InterruptedException {
+        try {
+            ZookeeperClient client = rtState.zkClient;
+
+            if (!client.exists(zkPaths.clusterDir)) {
+                createRootPathParents(zkPaths.clusterDir, client);
+
+                client.createIfNeeded(zkPaths.clusterDir, null, PERSISTENT);
+            }
+
+            List<String> createdDirs = client.getChildren(zkPaths.clusterDir);
+
+            String[] requiredDirs = {
+                zkPaths.evtsPath,
+                zkPaths.joinDataDir,
+                zkPaths.customEvtsDir,
+                zkPaths.customEvtsPartsDir,
+                zkPaths.customEvtsAcksDir,
+                zkPaths.aliveNodesDir};
+
+            List<String> dirs = new ArrayList<>();
+
+            for (String dir : requiredDirs) {
+                String dir0 = dir.substring(zkPaths.clusterDir.length() + 1);
+
+                if (!createdDirs.contains(dir0))
+                    dirs.add(dir);
+            }
+
+            try {
+                if (!dirs.isEmpty())
+                    client.createAll(dirs, PERSISTENT);
+            }
+            catch (KeeperException.NodeExistsException e) {
+                if (log.isDebugEnabled())
+                    log.debug("Failed to create nodes using bulk operation: " + e);
+
+                for (String dir : dirs)
+                    client.createIfNeeded(dir, null, PERSISTENT);
+            }
+        }
+        catch (ZookeeperClientFailedException e) {
+            throw new IgniteSpiException("Failed to initialize Zookeeper nodes", e);
+        }
+    }
+
+    /**
+     * @param rootDir Root directory.
+     * @param client Client.
+     * @throws ZookeeperClientFailedException If connection to zk was lost.
+     * @throws InterruptedException If interrupted.
+     */
+    private void createRootPathParents(String rootDir, ZookeeperClient client)
+        throws ZookeeperClientFailedException, InterruptedException {
+        int startIdx = 0;
+
+        for (;;) {
+            int separatorIdx = rootDir.indexOf(ZkIgnitePaths.PATH_SEPARATOR, startIdx);
+
+            if (separatorIdx == -1)
+                break;
+
+            if (separatorIdx > 0) {
+                String path = rootDir.substring(0, separatorIdx);
+
+                client.createIfNeeded(path, null, CreateMode.PERSISTENT);
+            }
+
+            startIdx = separatorIdx + 1;
+        }
+    }
+
+    /**
+     * @param zkClient Client.
+     * @param basePath Base path.
+     * @param partCnt Parts count.
+     */
+    private void deleteMultiplePartsAsync(ZookeeperClient zkClient, String basePath, int partCnt) {
+        for (int i = 0; i < partCnt; i++) {
+            String path = multipartPathName(basePath, i);
+
+            zkClient.deleteIfExistsAsync(path);
+        }
+    }
+
+    /**
+     * @param zkClient Client.
+     * @param basePath Base path.
+     * @param partCnt Parts count.
+     * @return Read parts.
+     * @throws Exception If failed.
+     */
+    private byte[] readMultipleParts(ZookeeperClient zkClient, String basePath, int partCnt)
+        throws Exception {
+        assert partCnt >= 1;
+
+        if (partCnt > 1) {
+            List<byte[]> parts = new ArrayList<>(partCnt);
+
+            int totSize = 0;
+
+            for (int i = 0; i < partCnt; i++) {
+                byte[] part = zkClient.getData(multipartPathName(basePath, i));
+
+                parts.add(part);
+
+                totSize += part.length;
+            }
+
+            byte[] res = new byte[totSize];
+
+            int pos = 0;
+
+            for (int i = 0; i < partCnt; i++) {
+                byte[] part = parts.get(i);
+
+                System.arraycopy(part, 0, res, pos, part.length);
+
+                pos += part.length;
+            }
+
+            return res;
+        }
+        else
+            return zkClient.getData(multipartPathName(basePath, 0));
+    }
+
+    /**
+     * @param zkClient Client.
+     * @param basePath Base path.
+     * @param parts Data parts.
+     * @return Number of parts.
+     * @throws ZookeeperClientFailedException If client failed.
+     * @throws InterruptedException If interrupted.
+     */
+    private int saveMultipleParts(ZookeeperClient zkClient, String basePath, List<byte[]> parts)
+        throws ZookeeperClientFailedException, InterruptedException
+    {
+        assert parts.size() > 1;
+
+        for (int i = 0; i < parts.size(); i++) {
+            byte[] part = parts.get(i);
+
+            String path = multipartPathName(basePath, i);
+
+            zkClient.createIfNeeded(path, part, PERSISTENT);
+        }
+
+        return parts.size();
+    }
+
+    /**
+     * @param basePath Base path.
+     * @param part Part number.
+     * @return Path.
+     */
+    private static String multipartPathName(String basePath, int part) {
+        return basePath + String.format("%04d", part);
+    }
+
+    /**
+     * @param rtState Runtime state.
+     * @param joinDataBytes Joining node data.
+     * @param prevState Previous state in case of connect retry.
+     * @throws InterruptedException If interrupted.
+     */
+    private void startJoin(ZkRuntimeState rtState, @Nullable ZkRuntimeState prevState, final byte[] joinDataBytes)
+        throws InterruptedException
+    {
+        try {
+            long startTime = System.currentTimeMillis();
+
+            initZkNodes();
+
+            String prefix = UUID.randomUUID().toString();
+
+            rtState.init(new ZkWatcher(rtState), new AliveNodeDataWatcher(rtState));
+
+            ZookeeperClient zkClient = rtState.zkClient;
+
+            final int OVERHEAD = 5;
+
+            // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8193
+            String joinDataPath = zkPaths.joinDataDir + "/" + prefix + ":" + locNode.id();
+
+            if (zkClient.needSplitNodeData(joinDataPath, joinDataBytes, OVERHEAD)) {
+                List<byte[]> parts = zkClient.splitNodeData(joinDataPath, joinDataBytes, OVERHEAD);
+
+                rtState.joinDataPartCnt = parts.size();
+
+                saveMultipleParts(zkClient, joinDataPath + ":", parts);
+
+                joinDataPath = zkClient.createIfNeeded(
+                    joinDataPath,
+                    marshalZip(new ZkJoiningNodeData(parts.size())),
+                    PERSISTENT);
+            }
+            else {
+                joinDataPath = zkClient.createIfNeeded(
+                    joinDataPath,
+                    joinDataBytes,
+                    PERSISTENT);
+            }
+
+            rtState.locNodeZkPath = zkClient.createSequential(
+                prefix,
+                zkPaths.aliveNodesDir,
+                zkPaths.aliveNodePathForCreate(prefix, locNode),
+                null,
+                EPHEMERAL_SEQUENTIAL);
+
+            rtState.internalOrder = ZkIgnitePaths.aliveInternalId(rtState.locNodeZkPath);
+
+            if (log.isInfoEnabled()) {
+                log.info("Node started join [nodeId=" + locNode.id() +
+                    ", instanceName=" + locNode.attribute(ATTR_IGNITE_INSTANCE_NAME) +
+                    ", zkSessionId=0x" + Long.toHexString(rtState.zkClient.zk().getSessionId()) +
+                    ", joinDataSize=" + joinDataBytes.length +
+                    (rtState.joinDataPartCnt > 1 ? (", joinDataPartCnt=" + rtState.joinDataPartCnt) : "") +
+                    ", consistentId=" + locNode.consistentId() +
+                    ", initTime=" + (System.currentTimeMillis() - startTime) +
+                    ", nodePath=" + rtState.locNodeZkPath + ']');
+            }
+
+            /*
+            If node can not join due to validation error this error is reported in join data,
+            As a minor optimization do not start watch join data immediately, but only if do not receive
+            join event after some timeout.
+             */
+            CheckJoinErrorWatcher joinErrorWatcher = new CheckJoinErrorWatcher(5000, joinDataPath, rtState);
+
+            rtState.joinErrTo = joinErrorWatcher.timeoutObj;
+
+            if (locNode.isClient() && spi.getJoinTimeout() > 0) {
+                ZkTimeoutObject joinTimeoutObj = prevState != null ? prevState.joinTo : null;
+
+                if (joinTimeoutObj == null) {
+                    joinTimeoutObj = new JoinTimeoutObject(spi.getJoinTimeout());
+
+                    spi.getSpiContext().addTimeoutObject(joinTimeoutObj);
+                }
+
+                rtState.joinTo = joinTimeoutObj;
+            }
+
+            if (!locNode.isClient())
+                zkClient.getChildrenAsync(zkPaths.aliveNodesDir, null, new CheckCoordinatorCallback(rtState));
+
+            zkClient.getDataAsync(zkPaths.evtsPath, rtState.watcher, rtState.watcher);
+
+            spi.getSpiContext().addTimeoutObject(rtState.joinErrTo);
+        }
+        catch (IgniteCheckedException | ZookeeperClientFailedException e) {
+            throw new IgniteSpiException("Failed to initialize Zookeeper nodes", e);
+        }
+    }
+
+    /**
+     * Authenticate local node.
+     *
+     * @param nodeAuth Authenticator.
+     * @param locCred Local security credentials for authentication.
+     * @throws IgniteSpiException If any error occurs.
+     */
+    private void localAuthentication(DiscoverySpiNodeAuthenticator nodeAuth, SecurityCredentials locCred){
+        assert nodeAuth != null;
+        assert locCred != null;
+
+        try {
+            SecurityContext subj = nodeAuth.authenticateNode(locNode, locCred);
+
+            // Note: exception message is checked in tests.
+            if (subj == null)
+                throw new IgniteSpiException("Authentication failed for local node.");
+
+            if (!(subj instanceof Serializable))
+                throw new IgniteSpiException("Authentication subject is not Serializable.");
+
+            Map<String, Object> attrs = new HashMap<>(locNode.attributes());
+
+            attrs.put(ATTR_SECURITY_SUBJECT_V2, U.marshal(marsh, subj));
+
+            locNode.setAttributes(attrs);
+        }
+        catch (Exception e) {
+            throw new IgniteSpiException("Failed to authenticate local node (will shutdown local node).", e);
+        }
+    }
+
+    /**
+     * @param node Node.
+     * @param zipBytes Zip-compressed marshalled security subject.
+     * @throws Exception If failed.
+     */
+    private void setNodeSecuritySubject(ZookeeperClusterNode node, byte[] zipBytes) throws Exception {
+        assert zipBytes != null;
+
+        Map<String, Object> attrs = new HashMap<>(node.getAttributes());
+
+        attrs.put(ATTR_SECURITY_SUBJECT_V2, unzip(zipBytes));
+
+        node.setAttributes(attrs);
+    }
+
+    /**
+     * @param node Node.
+     * @return Credentials.
+     * @throws IgniteCheckedException If failed to unmarshal.
+     */
+    private SecurityCredentials unmarshalCredentials(ZookeeperClusterNode node) throws Exception {
+        byte[] credBytes = (byte[])node.getAttributes().get(ATTR_SECURITY_CREDENTIALS);
+
+        if (credBytes == null)
+            return null;
+
+        return unmarshalZip(credBytes);
+    }
+
+    /**
+     * Marshalls credentials with discovery SPI marshaller (will replace attribute value).
+     *
+     * @param node Node to marshall credentials for.
+     * @throws IgniteSpiException If marshalling failed.
+     */
+    private void marshalCredentialsOnJoin(ZookeeperClusterNode node) throws IgniteSpiException {
+        try {
+            // Use security-unsafe getter.
+            Map<String, Object> attrs0 = node.getAttributes();
+
+            Object creds = attrs0.get(ATTR_SECURITY_CREDENTIALS);
+
+            if (creds != null) {
+                Map<String, Object> attrs = new HashMap<>(attrs0);
+
+                assert !(creds instanceof byte[]);
+
+                attrs.put(ATTR_SECURITY_CREDENTIALS, marshalZip(creds));
+
+                node.setAttributes(attrs);
+            }
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteSpiException("Failed to marshal node security credentials: " + node.id(), e);
+        }
+    }
+
+    /**
+     *
+     */
+    private class UpdateProcessedEventsTimeoutObject extends ZkTimeoutObject {
+        /** */
+        private final ZkRuntimeState rtState;
+
+        /**
+         * @param rtState Runtime state.
+         * @param timeout Timeout.
+         */
+        UpdateProcessedEventsTimeoutObject(ZkRuntimeState rtState, long timeout) {
+            super(timeout);
+
+            this.rtState = rtState;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onTimeout() {
+            runInWorkerThread(new ZkRunnable(rtState, ZookeeperDiscoveryImpl.this) {
+                @Override protected void run0() throws Exception {
+                    updateProcessedEventsOnTimeout(rtState, UpdateProcessedEventsTimeoutObject.this);
+                }
+            });
+        }
+    }
+
+    /**
+     *
+     */
+    private class JoinTimeoutObject extends ZkTimeoutObject {
+        /**
+         * @param timeout Timeout.
+         */
+        JoinTimeoutObject(long timeout) {
+            super(timeout);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onTimeout() {
+            if (cancelled || rtState.joined)
+                return;
+
+            runInWorkerThread(new Runnable() {
+                @Override public void run() {
+                    synchronized (stateMux) {
+                        if (cancelled || rtState.joined)
+                            return;
+
+                        if (connState == ConnectionState.STOPPED)
+                            return;
+
+                        connState = ConnectionState.STOPPED;
+                    }
+
+                    U.warn(log, "Failed to connect to cluster, either connection to ZooKeeper can not be established or there " +
+                        "are no alive server nodes (consider increasing 'joinTimeout' configuration  property) [" +
+                        "joinTimeout=" + spi.getJoinTimeout() + ']');
+
+                    // Note: exception message is checked in tests.
+                    onSegmented(new IgniteSpiException("Failed to connect to cluster within configured timeout"));
+                }
+            });
+        }
+    }
+
+    /**
+     *
+     */
+    private class CheckJoinErrorWatcher extends ZkAbstractWatcher implements AsyncCallback.DataCallback {
+        /** */
+        private final String joinDataPath;
+
+        /** */
+        private ZkTimeoutObject timeoutObj;
+
+        /**
+         * @param timeout Timeout.
+         * @param joinDataPath0 Node joined data path.
+         * @param rtState0 State.
+         */
+        CheckJoinErrorWatcher(long timeout, String joinDataPath0, ZkRuntimeState rtState0) {
+            super(rtState0, ZookeeperDiscoveryImpl.this);
+
+            this.joinDataPath = joinDataPath0;
+
+            timeoutObj = new ZkTimeoutObject(timeout) {
+                @Override public void onTimeout() {
+                    if (rtState.errForClose != null || rtState.joined)
+                        return;
+
+                    synchronized (stateMux) {
+                        if (connState != ConnectionState.STARTED)
+                            return;
+                    }
+
+                    rtState.zkClient.getDataAsync(joinDataPath,
+                        CheckJoinErrorWatcher.this,
+                        CheckJoinErrorWatcher.this);
+                }
+            };
+        }
+
+        /** {@inheritDoc} */
+        @Override public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
+            if (rc != 0)
+                return;
+
+            if (!onProcessStart())
+                return;
+
+            try {
+                Object obj = unmarshalZip(data);
+
+                if (obj instanceof ZkInternalJoinErrorMessage) {
+                    ZkInternalJoinErrorMessage joinErr = (ZkInternalJoinErrorMessage)obj;
+
+                    onSegmented(new IgniteSpiException(joinErr.err));
+                }
+
+                onProcessEnd();
+            }
+            catch (Throwable e) {
+                onProcessError(e);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void process0(WatchedEvent evt) {
+            if (rtState.errForClose != null || rtState.joined)
+                return;
+
+            if (evt.getType() == Event.EventType.NodeDataChanged)
+                rtState.zkClient.getDataAsync(evt.getPath(), this, this);
+        }
+    }
+
+    /**
+     * @param aliveNodes Alive nodes.
+     * @throws Exception If failed.
+     */
+    private void checkIsCoordinator(final List<String> aliveNodes) throws Exception {
+        assert !locNode.isClient();
+
+        TreeMap<Long, String> aliveSrvs = new TreeMap<>();
+
+        long locInternalOrder = rtState.internalOrder;
+
+        for (String aliveNodePath : aliveNodes) {
+            if (ZkIgnitePaths.aliveNodeClientFlag(aliveNodePath))
+                continue;
+
+            Long internalId = ZkIgnitePaths.aliveInternalId(aliveNodePath);
+
+            aliveSrvs.put(internalId, aliveNodePath);
+        }
+
+        assert !aliveSrvs.isEmpty();
+
+        Map.Entry<Long, String> crdE = aliveSrvs.firstEntry();
+
+        if (locInternalOrder == crdE.getKey())
+            onBecomeCoordinator(aliveNodes);
+        else {
+            assert aliveSrvs.size() > 1 : aliveSrvs;
+
+            Map.Entry<Long, String> prevE = aliveSrvs.floorEntry(locInternalOrder - 1);
+
+            assert prevE != null;
+
+            if (log.isInfoEnabled()) {
+                log.info("Discovery coordinator already exists, watch for previous server node [" +
+                    "locId=" + locNode.id() +
+                    ", watchPath=" + prevE.getValue() + ']');
+             }
+
+            PreviousNodeWatcher watcher = new ServerPreviousNodeWatcher(rtState);
+
+            rtState.zkClient.existsAsync(zkPaths.aliveNodesDir + "/" + prevE.getValue(), watcher, watcher);
+        }
+    }
+
+    /**
+     * @param aliveNodes Alive nodes.
+     * @throws Exception If failed.
+     */
+    private void checkClientsStatus(final List<String> aliveNodes) throws Exception {
+        assert locNode.isClient() : locNode;
+        assert rtState.joined;
+        assert rtState.evtsData != null;
+
+        TreeMap<Long, String> aliveClients = new TreeMap<>();
+
+        String srvPath = null;
+        Long srvInternalOrder = null;
+
+        long locInternalOrder = rtState.internalOrder;
+
+        for (String aliveNodePath : aliveNodes) {
+            Long internalId = ZkIgnitePaths.aliveInternalId(aliveNodePath);
+
+            if (ZkIgnitePaths.aliveNodeClientFlag(aliveNodePath))
+                aliveClients.put(internalId, aliveNodePath);
+            else {
+                if (srvInternalOrder == null || internalId < srvInternalOrder) {
+                    srvPath = aliveNodePath;
+                    srvInternalOrder = internalId;
+                }
+            }
+        }
+
+        assert !aliveClients.isEmpty();
+
+        Map.Entry<Long, String> oldest = aliveClients.firstEntry();
+
+        boolean oldestClient = locInternalOrder == oldest.getKey();
+
+        if (srvPath == null) {
+            if (oldestClient) {
+                Stat stat = new Stat();
+
+                ZkDiscoveryEventsData prevEvts = rtState.evtsData;
+
+                byte[] evtsBytes = rtState.zkClient.getData(zkPaths.evtsPath, stat);
+
+                assert evtsBytes.length > 0;
+
+                ZkDiscoveryEventsData newEvts = unmarshalZip(evtsBytes);
+
+                if (prevEvts.clusterId.equals(newEvts.clusterId)) {
+                    U.warn(log, "All server nodes failed, notify all clients [locId=" + locNode.id() + ']');
+
+                    generateNoServersEvent(newEvts, stat);
+                }
+                else
+                    U.warn(log, "All server nodes failed (received events from new cluster).");
+            }
+        }
+        else {
+            String watchPath;
+
+            if (oldestClient) {
+                watchPath = srvPath;
+
+                if (log.isInfoEnabled()) {
+                    log.info("Servers exists, watch for server node [locId=" + locNode.id() +
+                        ", watchPath=" + watchPath + ']');
+                }
+            }
+            else {
+                assert aliveClients.size() > 1 : aliveClients;
+
+                Map.Entry<Long, String> prevE = aliveClients.floorEntry(locInternalOrder - 1);
+
+                assert prevE != null;
+
+                watchPath = prevE.getValue();
+
+                if (log.isInfoEnabled()) {
+                    log.info("Servers exists, watch for previous node [locId=" + locNode.id() +
+                        ", watchPath=" + watchPath + ']');
+                }
+            }
+
+            PreviousNodeWatcher watcher = new ClientPreviousNodeWatcher(rtState);
+
+            rtState.zkClient.existsAsync(zkPaths.aliveNodesDir + "/" + watchPath, watcher, watcher);
+        }
+    }
+
+    /**
+     * @param evtsData Events data.
+     * @param evtsStat Events zookeeper state.
+     * @throws Exception If failed.
+     */
+    private void generateNoServersEvent(ZkDiscoveryEventsData evtsData, Stat evtsStat) throws Exception {
+        evtsData.evtIdGen++;
+
+        ZkDiscoveryCustomEventData evtData = new ZkDiscoveryCustomEventData(
+            evtsData.evtIdGen,
+            0L,
+            evtsData.topVer,
+            locNode.id(),
+            new ZkNoServersMessage(),
+            null);
+
+        Collection<ZookeeperClusterNode> nodesToAck = Collections.emptyList();
+
+        evtsData.addEvent(nodesToAck, evtData);
+
+        byte[] newEvtsBytes = marshalZip(evtsData);
+
+        try {
+            rtState.zkClient.setData(zkPaths.evtsPath, newEvtsBytes, evtsStat.getVersion());
+        }
+        catch (KeeperException.BadVersionException e) {
+            // Version can change if new cluster started and saved new events.
+            if (log.isDebugEnabled())
+                log.debug("Failed to save no servers message");
+        }
+    }
+
+    /**
+     * @param lastEvts Last events from previous coordinator.
+     * @throws Exception If failed.
+     */
+    private void previousCoordinatorCleanup(ZkDiscoveryEventsData lastEvts) throws Exception {
+        for (ZkDiscoveryEventData evtData : lastEvts.evts.values()) {
+            if (evtData instanceof ZkDiscoveryCustomEventData) {
+                ZkDiscoveryCustomEventData evtData0 = (ZkDiscoveryCustomEventData)evtData;
+
+                // It is possible previous coordinator failed before finished cleanup.
+                if (evtData0.msg instanceof ZkCommunicationErrorResolveFinishMessage) {
+                    try {
+                        ZkCommunicationErrorResolveFinishMessage msg =
+                            (ZkCommunicationErrorResolveFinishMessage)evtData0.msg;
+
+                        ZkCommunicationErrorResolveResult res = unmarshalZip(
+                            ZkDistributedCollectDataFuture.readResult(rtState.zkClient, zkPaths, msg.futId));
+
+                        deleteAliveNodes(res.killedNodes);
+                    }
+                    catch (KeeperException.NoNodeException ignore) {
+                        // No-op.
+                    }
+                }
+                else if (evtData0.resolvedMsg instanceof ZkForceNodeFailMessage)
+                    deleteAliveNode(((ZkForceNodeFailMessage)evtData0.resolvedMsg).nodeInternalId);
+            }
+        }
+    }
+
+    /**
+     * @param aliveNodes Alive nodes paths.
+     * @throws Exception If failed.
+     */
+    private void onBecomeCoordinator(List<String> aliveNodes) throws Exception {
+        ZkDiscoveryEventsData prevEvts = processNewEvents(rtState.zkClient.getData(zkPaths.evtsPath));
+
+        rtState.crd = true;
+
+        if (rtState.joined) {
+            if (log.isInfoEnabled())
+                log.info("Node is new discovery coordinator [locId=" + locNode.id() + ']');
+
+            assert locNode.order() > 0 : locNode;
+            assert rtState.evtsData != null;
+
+            previousCoordinatorCleanup(rtState.evtsData);
+
+            UUID futId = rtState.evtsData.communicationErrorResolveFutureId();
+
+            if (futId != null) {
+                if (log.isInfoEnabled()) {
+                    log.info("New discovery coordinator will handle already started cluster-wide communication " +
+                        "error resolve [reqId=" + futId + ']');
+                }
+
+                ZkCommunicationErrorProcessFuture fut = commErrProcFut.get();
+
+                ZkDistributedCollectDataFuture collectResFut = collectCommunicationStatusFuture(futId);
+
+                if (fut != null)
+                    fut.nodeResultCollectFuture(collectResFut);
+            }
+
+            for (ZkDiscoveryEventData evtData : rtState.evtsData.evts.values())
+                evtData.initRemainingAcks(rtState.top.nodesByOrder.values());
+
+            handleProcessedEvents("crd");
+        }
+        else {
+            String locAlivePath = rtState.locNodeZkPath.substring(rtState.locNodeZkPath.lastIndexOf('/') + 1);
+
+            deleteJoiningNodeData(locNode.id(),
+                ZkIgnitePaths.aliveNodePrefixId(locAlivePath),
+                rtState.joinDataPartCnt);
+
+            DiscoverySpiNodeAuthenticator nodeAuth = spi.getAuthenticator();
+
+            if (nodeAuth != null) {
+                try {
+                    if (log.isInfoEnabled()) {
+                        log.info("Node is first server node in cluster, try authenticate local node " +
+                            "[locId=" + locNode.id() + ']');
+                    }
+
+                    localAuthentication(nodeAuth, unmarshalCredentials(locNode));
+                }
+                catch (Exception e) {
+                    U.warn(log, "Local node authentication failed: " + e, e);
+
+                    onSegmented(e);
+
+                    // Stop any further processing.
+                    throw new ZookeeperClientFailedException("Local node authentication failed: " + e);
+                }
+            }
+
+            newClusterStarted(prevEvts);
+        }
+
+        rtState.zkClient.getChildrenAsync(zkPaths.aliveNodesDir, rtState.watcher, rtState.watcher);
+        rtState.zkClient.getChildrenAsync(zkPaths.customEvtsDir, rtState.watcher, rtState.watcher);
+
+        for (String alivePath : aliveNodes)
+            watchAliveNodeData(alivePath);
+    }
+
+    /**
+     * @param alivePath Node path.
+     */
+    private void watchAliveNodeData(String alivePath) {
+        assert rtState.locNodeZkPath != null;
+
+        String path = zkPaths.aliveNodesDir + "/" + alivePath;
+
+        if (!path.equals(rtState.locNodeZkPath))
+            rtState.zkClient.getDataAsync(path, rtState.aliveNodeDataWatcher, rtState.aliveNodeDataWatcher);
+    }
+
+    /**
+     * @param aliveNodes ZK nodes representing alive cluster nodes.
+     * @throws Exception If failed.
+     */
+    private void generateTopologyEvents(List<String> aliveNodes) throws Exception {
+        assert rtState.crd;
+
+        if (log.isInfoEnabled())
+            log.info("Process alive nodes change [alives=" + aliveNodes.size() + "]");
+
+        if (rtState.updateAlives) {
+            aliveNodes = rtState.zkClient.getChildren(zkPaths.aliveNodesDir);
+
+            rtState.updateAlives = false;
+        }
+
+        TreeMap<Long, String> alives = new TreeMap<>();
+
+        for (String child : aliveNodes) {
+            Long internalId = ZkIgnitePaths.aliveInternalId(child);
+
+            Object old = alives.put(internalId, child);
+
+            assert old == null;
+        }
+
+        TreeMap<Long, ZookeeperClusterNode> curTop = new TreeMap<>(rtState.top.nodesByOrder);
+
+        int newEvts = 0;
+
+        final int MAX_NEW_EVTS = IgniteSystemProperties.getInteger(IGNITE_ZOOKEEPER_DISCOVERY_SPI_MAX_EVTS, 100);
+
+        List<ZookeeperClusterNode> failedNodes = null;
+
+        for (Map.Entry<Long, ZookeeperClusterNode> e : rtState.top.nodesByInternalId.entrySet()) {
+            if (!alives.containsKey(e.getKey())) {
+                ZookeeperClusterNode failedNode = e.getValue();
+
+                if (failedNodes == null)
+                    failedNodes = new ArrayList<>();
+
+                failedNodes.add(failedNode);
+
+                generateNodeFail(curTop, failedNode);
+
+                newEvts++;
+
+                if (newEvts == MAX_NEW_EVTS) {
+                    saveAndProcessNewEvents();
+
+                    if (log.isInfoEnabled()) {
+                        log.info("Delay alive nodes change process, max event threshold reached [newEvts=" + newEvts +
+                            ", totalEvts=" + rtState.evtsData.evts.size() + ']');
+                    }
+
+                    handleProcessedEventsOnNodesFail(failedNodes);
+
+                    throttleNewEventsGeneration();
+
+                    rtState.zkClient.getChildrenAsync(zkPaths.aliveNodesDir, rtState.watcher, rtState.watcher);
+
+                    return;
+                }
+            }
+        }
+
+        // Process failures before processing join, otherwise conflicts are possible in case of fast node stop/re-start.
+        if (newEvts > 0) {
+            saveAndProcessNewEvents();
+
+            handleProcessedEventsOnNodesFail(failedNodes);
+
+            rtState.zkClient.getChildrenAsync(zkPaths.aliveNodesDir, rtState.watcher, rtState.watcher);
+
+            return;
+        }
+
+        generateJoinEvents(curTop, alives, MAX_NEW_EVTS);
+
+        if (failedNodes != null)
+            handleProcessedEventsOnNodesFail(failedNodes);
+    }
+
+    /**
+     * @param curTop Current topology.
+     * @param alives Alive znodes.
+     * @param MAX_NEW_EVTS Max event to process.
+     * @throws Exception If failed.
+     */
+    private void generateJoinEvents(TreeMap<Long, ZookeeperClusterNode> curTop,
+        TreeMap<Long, String> alives,
+        final int MAX_NEW_EVTS) throws Exception
+    {
+       ZkBulkJoinContext joinCtx = new ZkBulkJoinContext();
+
+       for (Map.Entry<Long, String> e : alives.entrySet()) {
+           Long internalId = e.getKey();
+
+           if (!rtState.top.nodesByInternalId.containsKey(internalId)) {
+               UUID rslvFutId = rtState.evtsData.communicationErrorResolveFutureId();
+
+               if (rslvFutId != null) {
+                   if (log.isInfoEnabled()) {
+                       log.info("Delay alive nodes change process while communication error resolve " +
+                           "is in progress [reqId=" + rslvFutId + ']');
+                   }
+
+                   break;
+               }
+
+               processJoinOnCoordinator(joinCtx, curTop, internalId, e.getValue());
+
+               if (joinCtx.nodes() == MAX_NEW_EVTS) {
+                   generateBulkJoinEvent(curTop, joinCtx);
+
+                   if (log.isInfoEnabled()) {
+                       log.info("Delay alive nodes change process, max event threshold reached [" +
+                           "newEvts=" + joinCtx.nodes() +
+                           ", totalEvts=" + rtState.evtsData.evts.size() + ']');
+                   }
+
+                   throttleNewEventsGeneration();
+
+                   rtState.zkClient.getChildrenAsync(zkPaths.aliveNodesDir, rtState.watcher, rtState.watcher);
+
+                   return;
+               }
+           }
+       }
+
+       if (joinCtx.nodes() > 0)
+           generateBulkJoinEvent(curTop, joinCtx);
+    }
+
+    /**
+     * @param curTop Current topology.
+     * @param joinCtx Joined nodes context.
+     * @throws Exception If failed.
+     */
+    private void generateBulkJoinEvent(TreeMap<Long, ZookeeperClusterNode> curTop, ZkBulkJoinContext joinCtx)
+        throws Exception
+    {
+        rtState.evtsData.evtIdGen++;
+
+        long evtId = rtState.evtsData.evtIdGen;
+
+        List<T2<ZkJoinedNodeEvtData, Map<Integer, Serializable>>> nodes = joinCtx.nodes;
+
+        assert nodes != null && nodes.size() > 0;
+
+        int nodeCnt = nodes.size();
+
+        List<ZkJoinedNodeEvtData> joinedNodes = new ArrayList<>(nodeCnt);
+
+        Map<Long, byte[]> discoDataMap = U.newHashMap(nodeCnt);
+        Map<Long, Long> dupDiscoData = null;
+
+        for (int i = 0; i < nodeCnt; i++) {
+            T2<ZkJoinedNodeEvtData, Map<Integer, Serializable>> nodeEvtData = nodes.get(i);
+
+            Map<Integer, Serializable> discoData = nodeEvtData.get2();
+
+            byte[] discoDataBytes = U.marshal(marsh, discoData);
+
+            Long dupDataNode = null;
+
+            for (Map.Entry<Long, byte[]> e : discoDataMap.entrySet()) {
+                if (Arrays.equals(discoDataBytes, e.getValue())) {
+                    dupDataNode = e.getKey();
+
+                    break;
+                }
+            }
+
+            long nodeTopVer = nodeEvtData.get1().topVer;
+
+            if (dupDataNode != null) {
+                if (dupDiscoData == null)
+                    dupDiscoData = new HashMap<>();
+
+                Long old = dupDiscoData.put(nodeTopVer, dupDataNode);
+
+                assert old == null : old;
+            }
+            else
+                discoDataMap.put(nodeTopVer, discoDataBytes);
+
+            joinedNodes.add(nodeEvtData.get1());
+        }
+
+        int overhead = 5;
+
+        ZkJoinEventDataForJoined dataForJoined = new ZkJoinEventDataForJoined(
+            new ArrayList<>(curTop.values()),
+            discoDataMap,
+            dupDiscoData);
+
+        byte[] dataForJoinedBytes = marshalZip(dataForJoined);
+
+        long addDataStart = System.currentTimeMillis();
+
+        int dataForJoinedPartCnt = saveData(zkPaths.joinEventDataPathForJoined(evtId),
+            dataForJoinedBytes,
+            overhead);
+
+        long addDataTime = System.currentTimeMillis() - addDataStart;
+
+        ZkDiscoveryNodeJoinEventData evtData = new ZkDiscoveryNodeJoinEventData(
+            evtId,
+            rtState.evtsData.topVer,
+            joinedNodes,
+            dataForJoinedPartCnt);
+
+        rtState.evtsData.addEvent(curTop.values(), evtData);
+
+        if (log.isInfoEnabled()) {
+            if (nodeCnt > 1) {
+                log.info("Generated NODE_JOINED bulk event [" +
+                    "nodeCnt=" + nodeCnt +
+                    ", dataForJoinedSize=" + dataForJoinedBytes.length +
+                    ", dataForJoinedPartCnt=" + dataForJoinedPartCnt +
+                    ", addDataTime=" + addDataTime +
+                    ", evt=" + evtData + ']');
+            }
+            else {
+                log.info("Generated NODE_JOINED event [" +
+                    "dataForJoinedSize=" + dataForJoinedBytes.length +
+                    ", dataForJoinedPartCnt=" + dataForJoinedPartCnt +
+                    ", addDataTime=" + addDataTime +
+                    ", evt=" + evtData + ']');
+            }
+        }
+
+        saveAndProcessNewEvents();
+    }
+
+    /**
+     *
+     */
+    private void throttleNewEventsGeneration() {
+        long delay = IgniteSystemProperties.getLong(IGNITE_ZOOKEEPER_DISCOVERY_SPI_EVTS_THROTTLE, 0);
+
+        if (delay > 0) {
+            if (log.isInfoEnabled())
+                log.info("Sleep delay before generate new events [delay=" + delay + ']');
+
+            try {
+                Thread.sleep(delay);
+            }
+            catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+            }
+        }
+    }
+
+    /**
+     * @param nodeId Node ID.
+     * @param prefixId Path prefix.
+     * @return Join data.
+     * @throws Exception If failed.
+     */
+    private ZkJoiningNodeData unmarshalJoinData(UUID nodeId, UUID prefixId) throws Exception {
+        String joinDataPath = zkPaths.joiningNodeDataPath(nodeId, prefixId);
+
+        byte[] joinData = rtState.zkClient.getData(joinDataPath);
+
+        Object dataObj = unmarshalZip(joinData);
+
+        if (!(dataObj instanceof ZkJoiningNodeData))
+            throw new Exception("Invalid joined node data: " + dataObj);
+
+        ZkJoiningNodeData joiningNodeData = (ZkJoiningNodeData)dataObj;
+
+        if (joiningNodeData.partCount() > 1) {
+            joinData = readMultipleParts(rtState.zkClient, joinDataPath + ":", joiningNodeData.partCount());
+
+            joiningNodeData = unmarshalZip(joinData);
+        }
+
+        return joiningNodeData;
+    }
+
+    /**
+     * @param nodeId Node ID.
+     * @param prefixId Path prefix.
+     * @param aliveNodePath Node path.
+     * @return Join data.
+     * @throws Exception If failed.
+     */
+    private Object unmarshalJoinDataOnCoordinator(UUID nodeId, UUID prefixId, String aliveNodePath) throws Exception {
+        String joinDataPath = zkPaths.joiningNodeDataPath(nodeId, prefixId);
+
+        byte[] joinData = rtState.zkClient.getData(joinDataPath);
+
+        Object dataObj;
+
+        try {
+            dataObj = unmarshalZip(joinData);
+
+            if (dataObj instanceof ZkInternalJoinErrorMessage)
+                return dataObj;
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to unmarshal joining node data [nodePath=" + aliveNodePath + "']", e);
+
+            return new ZkInternalJoinErrorMessage(ZkIgnitePaths.aliveInternalId(aliveNodePath),
+                "Failed to unmarshal join data: " + e);
+        }
+
+        assert dataObj instanceof ZkJoiningNodeData : dataObj;
+
+        ZkJoiningNodeData joiningNodeData = (ZkJoiningNodeData)dataObj;
+
+        if (joiningNodeData.partCount() > 1) {
+            joinData = readMultipleParts(rtState.zkClient, joinDataPath + ":", joiningNodeData.partCount());
+
+            try {
+                joiningNodeData = unmarshalZip(joinData);
+            }
+            catch (Exception e) {
+                U.error(log, "Failed to unmarshal joining node data [nodePath=" + aliveNodePath + "']", e);
+
+                return new ZkInternalJoinErrorMessage(ZkIgnitePaths.aliveInternalId(aliveNodePath),
+                    "Failed to unmarshal join data: " + e);
+            }
+        }
+
+        assert joiningNodeData.node() != null : joiningNodeData;
+
+        return joiningNodeData;
+    }
+
+    /**
+     * @param joinCtx Joined nodes context.
+     * @param curTop Current nodes.
+     * @param internalId Joined node internal ID.
+     * @param aliveNodePath Joined node path.
+     * @throws Exception If failed.
+     */
+    private void processJoinOnCoordinator(
+        ZkBulkJoinContext joinCtx,
+        TreeMap<Long, ZookeeperClusterNode> curTop,
+        long internalId,
+        String aliveNodePath)
+        throws Exception
+    {
+        UUID nodeId = ZkIgnitePaths.aliveNodeId(aliveNodePath);
+        UUID prefixId = ZkIgnitePaths.aliveNodePrefixId(aliveNodePath);
+
+        Object data = unmarshalJoinDataOnCoordinator(nodeId, prefixId, aliveNodePath);
+
+        if (data instanceof ZkJoiningNodeData) {
+            ZkJoiningNodeData joiningNodeData = (ZkJoiningNodeData)data;
+
+            ZkNodeValidateResult validateRes = validateJoiningNode(joiningNodeData.node());
+
+            if (validateRes.err == null) {
+                ZookeeperClusterNode joinedNode = joiningNodeData.node();
+
+                assert nodeId.equals(joinedNode.id()) : joiningNodeData.node();
+
+                addJoinedNode(
+                    joinCtx,
+                    curTop,
+                    joiningNodeData,
+                    internalId,
+                    prefixId,
+                    validateRes.secSubjZipBytes);
+
+                watchAliveNodeData(aliveNodePath);
+            }
+            else {
+                ZkInternalJoinErrorMessage joinErr = new ZkInternalJoinErrorMessage(
+                    ZkIgnitePaths.aliveInternalId(aliveNodePath),
+                    validateRes.err);
+
+                processJoinError(aliveNodePath, nodeId, prefixId, joinErr);
+            }
+        }
+        else {
+            assert data instanceof ZkInternalJoinErrorMessage : data;
+
+            processJoinError(aliveNodePath, nodeId, prefixId, (ZkInternalJoinErrorMessage)data);
+        }
+    }
+
+    /**
+     * @param aliveNodePath Joined node path.
+     * @param nodeId Node ID.
+     * @param prefixId Path prefix ID.
+     * @param joinErr Join error message.
+     * @throws Exception If failed.
+     */
+    private void processJoinError(String aliveNodePath,
+        UUID nodeId,
+        UUID prefixId,
+        ZkInternalJoinErrorMessage joinErr) throws Exception {
+        ZookeeperClient client = rtState.zkClient;
+
+        if (joinErr.notifyNode) {
+            String joinDataPath = zkPaths.joiningNodeDataPath(nodeId, prefixId);
+
+            client.setData(joinDataPath, marshalZip(joinErr), -1);
+
+            client.deleteIfExists(zkPaths.aliveNodesDir + "/" + aliveNodePath, -1);
+        }
+        else {
+            if (log.isInfoEnabled())
+                log.info("Ignore join data, node was failed by previous coordinator: " + aliveNodePath);
+
+            client.deleteIfExists(zkPaths.aliveNodesDir + "/" + aliveNodePath, -1);
+        }
+    }
+
+    /**
+     * @param node Joining node.
+     * @return Validation result.
+     */
+    private ZkNodeValidateResult validateJoiningNode(ZookeeperClusterNode node) {
+        ZookeeperClusterNode node0 = rtState.top.nodesById.get(node.id());
+
+        if (node0 != null) {
+            U.error(log, "Failed to include node in cluster, node with the same ID already exists [joiningNode=" + node +
+                ", existingNode=" + node0 + ']');
+
+            // Note: exception message is checked in tests.
+            return new ZkNodeValidateResult("Node with the same ID already exists: " + node0);
+        }
+
+        ZkNodeValidateResult res = authenticateNode(node);
+
+        if (res.err != null)
+            return res;
+
+        IgniteNodeValidationResult err = spi.getSpiContext().validateNode(node);
+
+        if (err != null) {
+            LT.warn(log, err.message());
+
+            res.err = err.sendMessage();
+        }
+
+        return res;
+    }
+
+    /**
+     * @param node Node.
+     * @return Validation result.
+     */
+    private ZkNodeValidateResult authenticateNode(ZookeeperClusterNode node) {
+        DiscoverySpiNodeAuthenticator nodeAuth = spi.getAuthenticator();
+
+        if (nodeAuth == null)
+            return new ZkNodeValidateResult((byte[])null);
+
+        SecurityCredentials cred;
+
+        try {
+            cred = unmarshalCredentials(node);
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to unmarshal node credentials: " + e, e);
+
+            return new ZkNodeValidateResult("Failed to unmarshal node credentials");
+        }
+
+        SecurityContext subj = nodeAuth.authenticateNode(node, cred);
+
+        if (subj == null) {
+            U.warn(log, "Authentication failed [nodeId=" + node.id() +
+                    ", addrs=" + U.addressesAsString(node) + ']',
+                "Authentication failed [nodeId=" + U.id8(node.id()) + ", addrs=" +
+                    U.addressesAsString(node) + ']');
+
+            // Note: exception message test is checked in tests.
+            return new ZkNodeValidateResult("Authentication failed");
+        }
+
+        if (!(subj instanceof Serializable)) {
+            U.warn(log, "Authentication subject is not Serializable [nodeId=" + node.id() +
+                    ", addrs=" + U.addressesAsString(node) + ']',
+                "Authentication subject is not Serializable [nodeId=" + U.id8(node.id()) +
+                    ", addrs=" +
+                    U.addressesAsString(node) + ']');
+
+            return new ZkNodeValidateResult("Authentication subject is not serializable");
+        }
+
+        byte[] secSubjZipBytes;
+
+        try {
+            secSubjZipBytes = marshalZip(subj);
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to marshal node security subject: " + e, e);
+
+            return new ZkNodeValidateResult("Failed to marshal node security subject");
+        }
+
+        return new ZkNodeValidateResult(secSubjZipBytes);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void saveAndProcessNewEvents() throws Exception {
+        if (stopping())
+            return;
+
+        long start = System.currentTimeMillis();
+
+        byte[] evtsBytes = marshalZip(rtState.evtsData);
+
+        rtState.zkClient.setData(zkPaths.evtsPath, evtsBytes, -1);
+
+        long time = System.currentTimeMillis() - start;
+
+        if (prevSavedEvtsTopVer != rtState.evtsData.topVer) {
+            if (log.isInfoEnabled()) {
+                log.info("Discovery coordinator saved new topology events [topVer=" + rtState.evtsData.topVer +
+                    ", size=" + evtsBytes.length +
+                    ", evts=" + rtState.evtsData.evts.size() +
+                    ", lastEvt=" + rtState.evtsData.evtIdGen +
+                    ", saveTime=" + time + ']');
+            }
+
+            prevSavedEvtsTopVer = rtState.evtsData.topVer;
+        }
+        else if (log.isDebugEnabled()) {
+            log.debug("Discovery coordinator saved new topology events [topVer=" + rtState.evtsData.topVer +
+                ", size=" + evtsBytes.length +
+                ", evts=" + rtState.evtsData.evts.size() +
+                ", lastEvt=" + rtState.evtsData.evtIdGen +
+                ", saveTime=" + time + ']');
+        }
+
+        processNewEvents(rtState.evtsData);
+    }
+
+    /**
+     * @param curTop Current topology.
+     * @param failedNode Failed node.
+     */
+    private void generateNodeFail(TreeMap<Long, ZookeeperClusterNode> curTop, ZookeeperClusterNode failedNode) {
+        Object rmvd = curTop.remove(failedNode.order());
+
+        assert rmvd != null;
+
+        rtState.evtsData.topVer++;
+        rtState.evtsData.evtIdGen++;
+
+        ZkDiscoveryNodeFailEventData evtData = new ZkDiscoveryNodeFailEventData(
+            rtState.evtsData.evtIdGen,
+            rtState.evtsData.topVer,
+            failedNode.internalId());
+
+        rtState.evtsData.addEvent(curTop.values(), evtData);
+
+        if (log.isInfoEnabled())
+            log.info("Generated NODE_FAILED event [evt=" + evtData + ']');
+    }
+
+    /**
+     * @param curTop Current nodes.
+     * @param joiningNodeData Join data.
+     * @param internalId Joined node internal ID.
+     * @param prefixId Unique path prefix.
+     * @param secSubjZipBytes Marshalled security subject.
+     * @throws Exception If failed.
+     */
+    private void addJoinedNode(
+        ZkBulkJoinContext joinCtx,
+        TreeMap<Long, ZookeeperClusterNode> curTop,
+        ZkJoiningNodeData joiningNodeData,
+        long internalId,
+        UUID prefixId,
+        @Nullable byte[] secSubjZipBytes)
+        throws Exception
+    {
+        ZookeeperClusterNode joinedNode = joiningNodeData.node();
+
+        UUID nodeId = joinedNode.id();
+
+        rtState.evtsData.topVer++;
+
+        joinedNode.order(rtState.evtsData.topVer);
+        joinedNode.internalId(internalId);
+
+        DiscoveryDataBag joiningNodeBag = new DiscoveryDataBag(nodeId, joiningNodeData.node().isClient());
+
+        joiningNodeBag.joiningNodeData(joiningNodeData.discoveryData());
+
+        exchange.onExchange(joiningNodeBag);
+
+        DiscoveryDataBag collectBag = new DiscoveryDataBag(nodeId,
+            new HashSet<Integer>(),
+            joiningNodeData.node().isClient());
+
+        collectBag.joiningNodeData(joiningNodeBag.joiningNodeData());
+
+        exchange.collect(collectBag);
+
+        Map<Integer, Serializable> commonData = collectBag.commonData();
+
+        Object old = curTop.put(joinedNode.order(), joinedNode);
+
+        assert old == null;
+
+        int overhead = 5;
+
+        int secSubjPartCnt = 0;
+
+        if (secSubjZipBytes != null) {
+            secSubjPartCnt = saveData(zkPaths.joinEventSecuritySubjectPath(joinedNode.order()),
+                secSubjZipBytes,
+                overhead);
+
+            assert secSubjPartCnt > 0 : secSubjPartCnt;
+
+            setNodeSecuritySubject(joinedNode, secSubjZipBytes);
+        }
+
+        ZkJoinedNodeEvtData nodeEvtData = new ZkJoinedNodeEvtData(
+            rtState.evtsData.topVer,
+            joinedNode.id(),
+            joinedNode.internalId(),
+            prefixId,
+            joiningNodeData.partCount(),
+            secSubjPartCnt);
+
+        nodeEvtData.joiningNodeData = joiningNodeData;
+
+        joinCtx.addJoinedNode(nodeEvtData, commonData);
+
+        rtState.evtsData.onNodeJoin(joinedNode);
+    }
+
+    /**
+     * @param path Path to save.
+     * @param bytes Bytes to save.
+     * @param overhead Extra overhead.
+     * @return Parts count.
+     * @throws Exception If failed.
+     */
+    private int saveData(String path, byte[] bytes, int overhead) throws Exception {
+        int dataForJoinedPartCnt = 1;
+
+        if (rtState.zkClient.needSplitNodeData(path, bytes, overhead)) {
+            dataForJoinedPartCnt = saveMultipleParts(rtState.zkClient,
+                path,
+                rtState.zkClient.splitNodeData(path, bytes, overhead));
+        }
+        else {
+            rtState.zkClient.createIfNeeded(multipartPathName(path, 0),
+                bytes,
+                PERSISTENT);
+        }
+
+        return dataForJoinedPartCnt;
+    }
+
+    /**
+     * @param prevEvts Events from previous cluster.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    private void newClusterStarted(@Nullable ZkDiscoveryEventsData prevEvts) throws Exception {
+        assert !locNode.isClient() : locNode;
+
+        long locInternalId = rtState.internalOrder;
+
+        assert prevEvts == null || prevEvts.maxInternalOrder < locInternalId;
+
+        spi.getSpiContext().removeTimeoutObject(rtState.joinErrTo);
+
+        cleanupPreviousClusterData(prevEvts != null ? prevEvts.maxInternalOrder + 1 : -1L);
+
+        rtState.joined = true;
+        rtState.gridStartTime = System.currentTimeMillis();
+
+        rtState.evtsData = ZkDiscoveryEventsData.createForNewCluster(rtState.gridStartTime);
+
+        if (log.isInfoEnabled()) {
+            log.info("New cluster started [locId=" + locNode.id() +
+                ", clusterId=" + rtState.evtsData.clusterId +
+                ", startTime=" + rtState.evtsData.clusterStartTime + ']');
+        }
+
+        locNode.internalId(locInternalId);
+        locNode.order(1);
+
+        rtState.evtsData.onNodeJoin(locNode);
+
+        rtState.top.addNode(locNode);
+
+        final List<ClusterNode> topSnapshot = Collections.singletonList((ClusterNode)locNode);
+
+        lsnr.onDiscovery(EVT_NODE_JOINED,
+            1L,
+            locNode,
+            topSnapshot,
+            Collections.<Long, Collection<ClusterNode>>emptyMap(),
+            null);
+
+        // Reset events (this is also notification for clients left from previous cluster).
+        rtState.zkClient.setData(zkPaths.evtsPath, marshalZip(rtState.evtsData), -1);
+
+        joinFut.onDone();
+    }
+
+    /**
+     * @param startInternalOrder Starting internal order for cluster (znodes having lower order belong
+     *      to clients from previous cluster and should be removed).
+
+     * @throws Exception If failed.
+     */
+    private void cleanupPreviousClusterData(long startInternalOrder) throws Exception {
+        long start = System.currentTimeMillis();
+
+        ZookeeperClient client = rtState.zkClient;
+
+        // TODO ZK: use multi, better batching + max-size safe + NoNodeException safe.
+        List<String> evtChildren = rtState.zkClient.getChildren(zkPaths.evtsPath);
+
+        for (String evtPath : evtChildren) {
+            String evtDir = zkPaths.evtsPath + "/" + evtPath;
+
+            removeChildren(evtDir);
+        }
+
+        client.deleteAll(zkPaths.evtsPath, evtChildren, -1);
+
+        client.deleteAll(zkPaths.customEvtsDir,
+            client.getChildren(zkPaths.customEvtsDir),
+            -1);
+
+        rtState.zkClient.deleteAll(zkPaths.customEvtsPartsDir,
+            rtState.zkClient.getChildren(zkPaths.customEvtsPartsDir),
+            -1);
+
+        rtState.zkClient.deleteAll(zkPaths.customEvtsAcksDir,
+            rtState.zkClient.getChildren(zkPaths.customEvtsAcksDir),
+            -1);
+
+        if (startInternalOrder > 0) {
+            for (String alive : rtState.zkClient.getChildren(zkPaths.aliveNodesDir)) {
+                if (ZkIgnitePaths.aliveInternalId(alive) < startInternalOrder)
+                    rtState.zkClient.deleteIfExists(zkPaths.aliveNodesDir + "/" + alive, -1);
+            }
+        }
+
+        long time = System.currentTimeMillis() - start;
+
+        if (time > 0) {
+            if (log.isInfoEnabled())
+                log.info("Previous cluster data cleanup time: " + time);
+        }
+    }
+
+    /**
+     * @param path Path.
+     * @throws Exception If failed.
+     */
+    private void removeChildren(String path) throws Exception {
+        rtState.zkClient.deleteAll(path, rtState.zkClient.getChildren(path), -1);
+    }
+
+    /**
+     * @param zkClient Client.
+     * @param evtPath Event path.
+     * @param sndNodeId Sender node ID.
+     * @return Event data.
+     * @throws Exception If failed.
+     */
+    private byte[] readCustomEventData(ZookeeperClient zkClient, String evtPath, UUID sndNodeId) throws Exception {
+        int partCnt = ZkIgnitePaths.customEventPartsCount(evtPath);
+
+        if (partCnt > 1) {
+            String partsBasePath = zkPaths.customEventPartsBasePath(
+                ZkIgnitePaths.customEventPrefix(evtPath), sndNodeId);
+
+            return readMultipleParts(zkClient, partsBasePath, partCnt);
+        }
+        else
+            return zkClient.getData(zkPaths.customEvtsDir + "/" + evtPath);
+    }
+
+    /**
+     * @param customEvtNodes ZK nodes representing custom events to process.
+     * @throws Exception If failed.
+     */
+    private void generateCustomEvents(List<String> customEvtNodes) throws Exception {
+        assert rtState.crd;
+
+        ZookeeperClient zkClient = rtState.zkClient;
+        ZkDiscoveryEventsData evtsData = rtState.evtsData;
+
+        TreeMap<Integer, String> unprocessedEvts = null;
+
+        for (int i = 0; i < customEvtNodes.size(); i++) {
+            String evtPath = customEvtNodes.get(i);
+
+            int evtSeq = ZkIgnitePaths.customEventSequence(evtPath);
+
+            if (evtSeq > evtsData.procCustEvt) {
+                if (unprocessedEvts == null)
+                    unprocessedEvts = new TreeMap<>();
+
+                unprocessedEvts.put(evtSeq, evtPath);
+            }
+        }
+
+        if (unprocessedEvts == null)
+            return;
+
+        for (Map.Entry<Integer, String> evtE : unprocessedEvts.entrySet()) {
+            evtsData.procCustEvt = evtE.getKey();
+
+            String evtPath = evtE.getValue();
+
+            UUID sndNodeId = ZkIgnitePaths.customEventSendNodeId(evtPath);
+
+            ZookeeperClusterNode sndNode = rtState.top.nodesById.get(sndNodeId);
+
+            if (sndNode != null) {
+                byte[] evtBytes = readCustomEventData(zkClient, evtPath, sndNodeId);
+
+                DiscoverySpiCustomMessage msg;
+
+                try {
+                    msg = unmarshalZip(evtBytes);
+                }
+                catch (Exception e) {
+                    U.error(log, "Failed to unmarshal custom discovery message: " + e, e);
+
+                    deleteCustomEventDataAsync(rtState.zkClient, evtPath);
+
+                    continue;
+                }
+
+                generateAndProcessCustomEventOnCoordinator(evtPath, sndNode, msg);
+            }
+            else {
+                U.warn(log, "Ignore custom event from unknown node: " + sndNodeId);
+
+                deleteCustomEventDataAsync(rtState.zkClient, evtPath);
+            }
+        }
+    }
+
+    /**
+     * @param evtPath Event data path.
+     * @param sndNode Sender node.
+     * @param msg Message instance.
+     * @throws Exception If failed.
+     */
+    private void generateAndProcessCustomEventOnCoordinator(String evtPath,
+        ZookeeperClusterNode sndNode,
+        DiscoverySpiCustomMessage msg) throws Exception
+    {
+        ZookeeperClient zkClient = rtState.zkClient;
+        ZkDiscoveryEventsData evtsData = rtState.evtsData;
+
+        ZookeeperClusterNode failedNode = null;
+
+        if (msg instanceof ZkForceNodeFailMessage) {
+            ZkForceNodeFailMessage msg0 = (ZkForceNodeFailMessage)msg;
+
+            failedNode = rtState.top.nodesByInternalId.get(msg0.nodeInternalId);
+
+            if (failedNode != null)
+                evtsData.topVer++;
+            else {
+                if (log.isDebugEnabled())
+                    log.debug("Ignore forcible node fail request for unknown node: " + msg0.nodeInternalId);
+
+                deleteCustomEventDataAsync(zkClient, evtPath);
+
+                return;
+            }
+        }
+        else if (msg instanceof ZkCommunicationErrorResolveStartMessage) {
+            ZkCommunicationErrorResolveStartMessage msg0 =
+                (ZkCommunicationErrorResolveStartMessage)msg;
+
+            if (evtsData.communicationErrorResolveFutureId() != null) {
+                if (log.isInfoEnabled()) {
+                    log.info("Ignore communication error resolve message, resolve process " +
+                        "already started [sndNode=" + sndNode + ']');
+                }
+
+                deleteCustomEventDataAsync(zkClient, evtPath);
+
+                return;
+            }
+            else {
+                if (log.isInfoEnabled()) {
+                    log.info("Start cluster-wide communication error resolve [sndNode=" + sndNode +
+                        ", reqId=" + msg0.id +
+                        ", topVer=" + evtsData.topVer + ']');
+                }
+
+                zkClient.createIfNeeded(zkPaths.distributedFutureBasePath(msg0.id),
+                    null,
+                    PERSISTENT);
+
+                evtsData.communicationErrorResolveFutureId(msg0.id);
+            }
+        }
+
+        evtsData.evtIdGen++;
+
+        ZkDiscoveryCustomEventData evtData = new ZkDiscoveryCustomEventData(
+            evtsData.evtIdGen,
+            0L,
+            evtsData.topVer,
+            sndNode.id(),
+            null,
+            evtPath);
+
+        evtData.resolvedMsg = msg;
+
+        if (log.isDebugEnabled())
+            log.debug("Generated CUSTOM event [evt=" + evtData + ", msg=" + msg + ']');
+
+        boolean fastStopProcess = false;
+
+        if (msg instanceof ZkInternalMessage)
+            processInternalMessage(evtData, (ZkInternalMessage)msg);
+        else {
+            notifyCustomEvent(evtData, msg);
+
+            if (msg.stopProcess()) {
+                if (log.isDebugEnabled())
+                    log.debug("Fast stop process custom event [evt=" + evtData + ", msg=" + msg + ']');
+
+                fastStopProcess = true;
+
+                // No need to process this event on others nodes, skip this event.
+                evtsData.evts.remove(evtData.eventId());
+
+                evtsData.evtIdGen--;
+
+                DiscoverySpiCustomMessage ack = msg.ackMessage();
+
+                if (ack != null) {
+                    evtData = createAckEvent(ack, evtData);
+
+                    if (log.isDebugEnabled())
+                        log.debug("Generated CUSTOM event (ack for fast stop process) [evt=" + evtData + ", msg=" + msg + ']');
+
+                    notifyCustomEvent(evtData, ack);
+                }
+                else
+                    evtData = null;
+            }
+        }
+
+        if (evtData != null) {
+            evtsData.addEvent(rtState.top.nodesByOrder.values(), evtData);
+
+            rtState.locNodeInfo.lastProcEvt = evtData.eventId();
+
+            saveAndProcessNewEvents();
+
+            if (fastStopProcess)
+                deleteCustomEventDataAsync(zkClient, evtPath);
+
+            if (failedNode != null) {
+                deleteAliveNode(failedNode.internalId());
+
+                handleProcessedEventsOnNodesFail(Collections.singletonList(failedNode));
+
+                rtState.updateAlives = true;
+            }
+        }
+    }
+
+    /**
+     * @param internalId Node internal ID.
+     * @throws Exception If failed.
+     */
+    private void deleteAliveNode(long internalId) throws Exception {
+        for (String child : rtState.zkClient.getChildren(zkPaths.aliveNodesDir)) {
+            if (ZkIgnitePaths.aliveInternalId(child) == internalId) {
+                // Need use sync delete to do not process again join of this node again.
+                rtState.zkClient.deleteIfExists(zkPaths.aliveNodesDir + "/" + child, -1);
+
+                return;
+            }
+        }
+    }
+
+    /**
+     * @param zkClient Client.
+     * @param evtPath Event path.
+     */
+    private void deleteCustomEventDataAsync(ZookeeperClient zkClient, String evtPath) {
+        if (log.isDebugEnabled())
+            log.debug("Delete custom event data: " + evtPath);
+
+        String prefix = ZkIgnitePaths.customEventPrefix(evtPath);
+        UUID sndNodeId = ZkIgnitePaths.customEventSendNodeId(evtPath);
+        int partCnt = ZkIgnitePaths.customEventPartsCount(evtPath);
+
+        assert partCnt >= 1 : partCnt;
+
+        if (partCnt > 1) {
+            for (int i = 0; i < partCnt; i++) {
+                String path0 = zkPaths.customEventPartPath(prefix, sndNodeId, i);
+
+                zkClient.deleteIfExistsAsync(path0);
+            }
+        }
+
+        zkClient.deleteIfExistsAsync(zkPaths.customEvtsDir + "/" + evtPath);
+    }
+
+    /**
+     * @param data Marshalled events.
+     * @throws Exception If failed.
+     * @return Events.
+     */
+    @Nullable private ZkDiscoveryEventsData processNewEvents(byte[] data) throws Exception {
+        ZkDiscoveryEventsData newEvts = data.length > 0 ? (ZkDiscoveryEventsData)unmarshalZip(data) : null;
+
+        if (rtState.joined && (newEvts == null || !rtState.evtsData.clusterId.equals(newEvts.clusterId))) {
+            assert locNode.isClient() : locNode;
+
+            throw localNodeFail("All server nodes failed, client node disconnected (received events from new custer) " +
+                "[locId=" + locNode.id() + ']', true);
+        }
+
+        if (newEvts == null)
+            return null;
+
+        assert !rtState.crd;
+
+        // Need keep processed custom events since they contain message object which is needed to create ack.
+        if (!locNode.isClient() && rtState.evtsData != null) {
+            for (Map.Entry<Long, ZkDiscoveryEventData> e : rtState.evtsData.evts.entrySet()) {
+                ZkDiscoveryEventData evtData = e.getValue();
+
+                if (evtData.eventType() == ZkDiscoveryEventData.ZK_EVT_CUSTOM_EVT) {
+                    ZkDiscoveryCustomEventData evtData0 =
+                        (ZkDiscoveryCustomEventData)newEvts.evts.get(evtData.eventId());
+
+                    if (evtData0 != null)
+                        evtData0.resolvedMsg = ((ZkDiscoveryCustomEventData)evtData).resolvedMsg;
+                }
+            }
+        }
+
+        processNewEvents(newEvts);
+
+        if (rtState.joined)
+            rtState.evtsData = newEvts;
+
+        return newEvts;
+    }
+
+    /**
+     * @param evtsData Ev

<TRUNCATED>

[06/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveStartMessage.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveStartMessage.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveStartMessage.java
new file mode 100644
index 0000000..0c79c36
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveStartMessage.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.spi.discovery.zk.internal;
+
+import java.util.UUID;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Zk Communication Error Resolve Start Message.
+ */
+public class ZkCommunicationErrorResolveStartMessage implements DiscoverySpiCustomMessage, ZkInternalMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    final UUID id;
+
+    /**
+     * @param id Unique ID.
+     */
+    ZkCommunicationErrorResolveStartMessage(UUID id) {
+        this.id = id;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public DiscoverySpiCustomMessage ackMessage() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isMutable() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(ZkCommunicationErrorResolveStartMessage.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationFailureContext.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationFailureContext.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationFailureContext.java
new file mode 100644
index 0000000..d27b717
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationFailureContext.java
@@ -0,0 +1,188 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.CommunicationFailureContext;
+import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ *
+ */
+class ZkCommunicationFailureContext implements CommunicationFailureContext {
+    /** */
+    private static final Comparator<ClusterNode> NODE_ORDER_CMP = new Comparator<ClusterNode>() {
+        @Override public int compare(ClusterNode node1, ClusterNode node2) {
+            return Long.compare(node1.order(), node2.order());
+        }
+    };
+
+    /** */
+    private Set<ClusterNode> killedNodes = new HashSet<>();
+
+    /** */
+    private final Map<UUID, BitSet> nodesState;
+
+    /** */
+    private final List<ClusterNode> initialNodes;
+
+    /** */
+    private final List<ClusterNode> curNodes;
+
+    /** */
+    private final GridCacheSharedContext<?, ?> ctx;
+
+    /**
+     * @param ctx Context.
+     * @param curNodes Current topology snapshot.
+     * @param initialNodes Topology snapshot when communication error resolve started.
+     * @param nodesState Nodes communication state.
+     */
+    ZkCommunicationFailureContext(
+        GridCacheSharedContext<?, ?> ctx,
+        List<ClusterNode> curNodes,
+        List<ClusterNode> initialNodes,
+        Map<UUID, BitSet> nodesState)
+    {
+        this.ctx = ctx;
+        this.curNodes = Collections.unmodifiableList(curNodes);
+        this.initialNodes = initialNodes;
+        this.nodesState = nodesState;
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<ClusterNode> topologySnapshot() {
+        return curNodes;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean connectionAvailable(ClusterNode node1, ClusterNode node2) {
+        BitSet nodeState = nodesState.get(node1.id());
+
+        if (nodeState == null)
+            throw new IllegalArgumentException("Invalid node: " + node1);
+
+        int nodeIdx = Collections.binarySearch(initialNodes, node2, NODE_ORDER_CMP);
+
+        if (nodeIdx < 0)
+            throw new IllegalArgumentException("Invalid node: " + node2);
+
+        assert nodeIdx < nodeState.size() : nodeIdx;
+
+        return nodeState.get(nodeIdx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<String, CacheConfiguration<?, ?>> startedCaches() {
+        Map<Integer, DynamicCacheDescriptor> cachesMap = ctx.affinity().caches();
+
+        Map<String, CacheConfiguration<?, ?>> res = U.newHashMap(cachesMap.size());
+
+        for (DynamicCacheDescriptor desc : cachesMap.values()) {
+            if (desc.cacheType().userCache())
+                res.put(desc.cacheName(), desc.cacheConfiguration());
+        }
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<List<ClusterNode>> cacheAffinity(String cacheName) {
+        if (cacheName == null)
+            throw new NullPointerException("Null cache name.");
+
+        DynamicCacheDescriptor cacheDesc = ctx.affinity().caches().get(CU.cacheId(cacheName));
+
+        if (cacheDesc == null)
+            throw new IllegalArgumentException("Invalid cache name: " + cacheName);
+
+        GridAffinityAssignmentCache aff = ctx.affinity().groupAffinity(cacheDesc.groupId());
+
+        assert aff != null : cacheName;
+
+        return aff.readyAssignments(aff.lastVersion());
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<List<ClusterNode>> cachePartitionOwners(String cacheName) {
+        if (cacheName == null)
+            throw new NullPointerException("Null cache name.");
+
+        DynamicCacheDescriptor cacheDesc = ctx.affinity().caches().get(CU.cacheId(cacheName));
+
+        if (cacheDesc == null)
+            throw new IllegalArgumentException("Invalid cache name: " + cacheName);
+
+        if (cacheDesc.cacheConfiguration().getCacheMode() == CacheMode.LOCAL)
+            return Collections.emptyList();
+
+        CacheGroupContext grp = ctx.cache().cacheGroup(cacheDesc.groupId());
+
+        GridDhtPartitionTopology top;
+
+        if (grp == null) {
+            top = ctx.exchange().clientTopologyIfExists(cacheDesc.groupId());
+
+            assert top != null : cacheName;
+        }
+        else
+            top = grp.topology();
+
+        return top.allOwners();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void killNode(ClusterNode node) {
+        if (node == null)
+            throw new NullPointerException();
+
+        if (Collections.binarySearch(curNodes, node, NODE_ORDER_CMP) < 0)
+            throw new IllegalArgumentException("Invalid node: " + node);
+
+        killedNodes.add(node);
+    }
+
+    /**
+     * @return Nodes to fail.
+     */
+    Set<ClusterNode> killedNodes() {
+        return killedNodes;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "ZkCommunicationFailureContext []";
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryCustomEventData.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryCustomEventData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryCustomEventData.java
new file mode 100644
index 0000000..21dfe62
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryCustomEventData.java
@@ -0,0 +1,89 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.UUID;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+
+/**
+ *
+ */
+class ZkDiscoveryCustomEventData extends ZkDiscoveryEventData {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    final long origEvtId;
+
+    /** */
+    final UUID sndNodeId;
+
+    /** */
+    final String evtPath;
+
+    /** Message instance (can be marshalled as part of ZkDiscoveryCustomEventData or stored in separate znode. */
+    DiscoverySpiCustomMessage msg;
+
+    /** Unmarshalled message. */
+    transient DiscoverySpiCustomMessage resolvedMsg;
+
+    /**
+     * @param evtId Event ID.
+     * @param origEvtId For acknowledge events ID of original event.
+     * @param topVer Topology version.
+     * @param sndNodeId Sender node ID.
+     * @param msg Message instance.
+     * @param evtPath Event path.
+     */
+    ZkDiscoveryCustomEventData(
+        long evtId,
+        long origEvtId,
+        long topVer,
+        UUID sndNodeId,
+        DiscoverySpiCustomMessage msg,
+        String evtPath)
+    {
+        super(evtId, ZK_EVT_CUSTOM_EVT, topVer);
+
+        assert sndNodeId != null;
+        assert msg != null || origEvtId != 0 || !F.isEmpty(evtPath);
+
+        this.origEvtId = origEvtId;
+        this.msg = msg;
+        this.sndNodeId = sndNodeId;
+        this.evtPath = evtPath;
+    }
+
+    /**
+     * @return {@code True} for custom event ack message.
+     */
+    boolean ackEvent() {
+        return origEvtId != 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "ZkDiscoveryCustomEventData [" +
+            "evtId=" + eventId() +
+            ", topVer=" + topologyVersion() +
+            ", sndNode=" + sndNodeId +
+            ", ack=" + ackEvent() +
+            ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventData.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventData.java
new file mode 100644
index 0000000..d667a17
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventData.java
@@ -0,0 +1,165 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Set;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ *
+ */
+abstract class ZkDiscoveryEventData implements Serializable {
+    /** */
+    static final byte ZK_EVT_NODE_JOIN = 1;
+
+    /** */
+    static final byte ZK_EVT_NODE_FAILED = 2;
+
+    /** */
+    static final byte ZK_EVT_CUSTOM_EVT = 3;
+
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final long evtId;
+
+    /** */
+    private final byte evtType;
+
+    /** */
+    private final long topVer;
+
+    /** */
+    private transient Set<Long> remainingAcks;
+
+    /** */
+    int flags;
+
+    /**
+     * @param evtId Event ID.
+     * @param evtType Event type.
+     * @param topVer Topology version.
+     */
+    ZkDiscoveryEventData(long evtId, byte evtType, long topVer) {
+        assert evtType == ZK_EVT_NODE_JOIN || evtType == ZK_EVT_NODE_FAILED || evtType == ZK_EVT_CUSTOM_EVT : evtType;
+
+        this.evtId = evtId;
+        this.evtType = evtType;
+        this.topVer = topVer;
+    }
+
+    /**
+     * @param nodes Current nodes in topology.
+     */
+    void initRemainingAcks(Collection<ZookeeperClusterNode> nodes) {
+        assert remainingAcks == null : this;
+
+        remainingAcks = U.newHashSet(nodes.size());
+
+        for (ZookeeperClusterNode node : nodes) {
+            if (!node.isLocal() && node.order() <= topVer) {
+                boolean add = remainingAcks.add(node.internalId());
+
+                assert add : node;
+            }
+        }
+    }
+
+    /**
+     * @param node Node.
+     */
+    void addRemainingAck(ZookeeperClusterNode node) {
+        assert node.order() <= topVer : node;
+
+        boolean add = remainingAcks.add(node.internalId());
+
+        assert add : node;
+    }
+
+    /**
+     * @return {@code True} if all nodes processed event.
+     */
+    boolean allAcksReceived() {
+        return remainingAcks.isEmpty();
+    }
+
+    /**
+     * @return Remaining acks.
+     */
+    Set<Long> remainingAcks() {
+        return remainingAcks;
+    }
+
+    /**
+     * @param nodeInternalId Node ID.
+     * @param ackEvtId Last event ID processed on node.
+     * @return {@code True} if all nodes processed event.
+     */
+    boolean onAckReceived(Long nodeInternalId, long ackEvtId) {
+        assert remainingAcks != null;
+
+        if (ackEvtId >= evtId)
+            remainingAcks.remove(nodeInternalId);
+
+        return remainingAcks.isEmpty();
+    }
+
+    /**
+     * @param node Failed node.
+     * @return {@code True} if all nodes processed event.
+     */
+    boolean onNodeFail(ZookeeperClusterNode node) {
+        assert remainingAcks != null : this;
+
+        remainingAcks.remove(node.internalId());
+
+        return remainingAcks.isEmpty();
+    }
+
+    /**
+     * @param flag Flag mask.
+     * @return {@code True} if flag set.
+     */
+    boolean flagSet(int flag) {
+        return (flags & flag) == flag;
+    }
+
+    /**
+     * @return Event ID.
+     */
+    long eventId() {
+        return evtId;
+    }
+
+    /**
+     * @return Event type.
+     */
+    byte eventType() {
+        return evtType;
+    }
+
+    /**
+     * @return Event topology version.
+     */
+    long topologyVersion() {
+        return topVer;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventsData.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventsData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventsData.java
new file mode 100644
index 0000000..dce861b
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventsData.java
@@ -0,0 +1,121 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.TreeMap;
+import java.util.UUID;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+class ZkDiscoveryEventsData implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Unique cluster ID (generated when first node in cluster starts). */
+    final UUID clusterId;
+
+    /** Internal order of last processed custom event. */
+    long procCustEvt = -1;
+
+    /** Event ID counter. */
+    long evtIdGen;
+
+    /** Current topology version. */
+    long topVer;
+
+    /** Max node internal order in cluster. */
+    long maxInternalOrder;
+
+    /** Cluster start time (recorded when first node in cluster starts). */
+    final long clusterStartTime;
+
+    /** Events to process. */
+    final TreeMap<Long, ZkDiscoveryEventData> evts;
+
+    /** ID of current active communication error resolve process. */
+    private UUID commErrFutId;
+
+    /**
+     * @param clusterStartTime Start time of first node in cluster.
+     * @return Events.
+     */
+    static ZkDiscoveryEventsData createForNewCluster(long clusterStartTime) {
+        return new ZkDiscoveryEventsData(
+            UUID.randomUUID(),
+            clusterStartTime,
+            1L,
+            new TreeMap<Long, ZkDiscoveryEventData>()
+        );
+    }
+
+    /**
+     * @param clusterId Cluster ID.
+     * @param topVer Current topology version.
+     * @param clusterStartTime Cluster start time.
+     * @param evts Events history.
+     */
+    private ZkDiscoveryEventsData(
+        UUID clusterId,
+        long clusterStartTime,
+        long topVer,
+        TreeMap<Long, ZkDiscoveryEventData> evts)
+    {
+        this.clusterId = clusterId;
+        this.clusterStartTime = clusterStartTime;
+        this.topVer = topVer;
+        this.evts = evts;
+    }
+
+    /**
+     * @param node Joined node.
+     */
+    void onNodeJoin(ZookeeperClusterNode node) {
+        if (node.internalId() > maxInternalOrder)
+            maxInternalOrder = node.internalId();
+    }
+
+    /**
+     * @return Future ID.
+     */
+    @Nullable UUID communicationErrorResolveFutureId() {
+        return commErrFutId;
+    }
+
+    /**
+     * @param id Future ID.
+     */
+     void communicationErrorResolveFutureId(@Nullable UUID id) {
+        commErrFutId = id;
+    }
+
+    /**
+     * @param nodes Current nodes in topology (these nodes should ack that event processed).
+     * @param evt Event.
+     */
+    void addEvent(Collection<ZookeeperClusterNode> nodes, ZkDiscoveryEventData evt) {
+        Object old = evts.put(evt.eventId(), evt);
+
+        assert old == null : old;
+
+        evt.initRemainingAcks(nodes);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeFailEventData.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeFailEventData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeFailEventData.java
new file mode 100644
index 0000000..c76158f
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeFailEventData.java
@@ -0,0 +1,55 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+/**
+ *
+ */
+class ZkDiscoveryNodeFailEventData extends ZkDiscoveryEventData {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private long failedNodeInternalId;
+
+    /**
+     * @param evtId Event ID.
+     * @param topVer Topology version.
+     * @param failedNodeInternalId Failed node ID.
+     */
+    ZkDiscoveryNodeFailEventData(long evtId, long topVer, long failedNodeInternalId) {
+        super(evtId, ZK_EVT_NODE_FAILED, topVer);
+
+        this.failedNodeInternalId = failedNodeInternalId;
+    }
+
+    /**
+     * @return Failed node ID.
+     */
+    long failedNodeInternalId() {
+        return failedNodeInternalId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "ZkDiscoveryNodeFailEventData [" +
+            "evtId=" + eventId() +
+            ", topVer=" + topologyVersion() +
+            ", nodeId=" + failedNodeInternalId + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeJoinEventData.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeJoinEventData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeJoinEventData.java
new file mode 100644
index 0000000..e46d52d
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeJoinEventData.java
@@ -0,0 +1,60 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.List;
+
+/**
+ *
+ */
+class ZkDiscoveryNodeJoinEventData extends ZkDiscoveryEventData {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    final List<ZkJoinedNodeEvtData> joinedNodes;
+
+    /** */
+    final int dataForJoinedPartCnt;
+
+    /**
+     * @param evtId Event ID.
+     * @param topVer Topology version.
+     * @param joinedNodes Joined nodes data.
+     * @param dataForJoinedPartCnt Data for joined part count.
+     */
+    ZkDiscoveryNodeJoinEventData(
+        long evtId,
+        long topVer,
+        List<ZkJoinedNodeEvtData> joinedNodes,
+        int dataForJoinedPartCnt)
+    {
+        super(evtId, ZK_EVT_NODE_JOIN, topVer);
+
+        this.joinedNodes = joinedNodes;
+        this.dataForJoinedPartCnt = dataForJoinedPartCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "ZkDiscoveryNodeJoinEventData [" +
+            "evtId=" + eventId() +
+            ", topVer=" + topologyVersion() +
+            ", nodes=" + joinedNodes + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDistributedCollectDataFuture.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDistributedCollectDataFuture.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDistributedCollectDataFuture.java
new file mode 100644
index 0000000..174d698
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDistributedCollectDataFuture.java
@@ -0,0 +1,250 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ *
+ */
+class ZkDistributedCollectDataFuture extends GridFutureAdapter<Void> {
+    /** */
+    private final IgniteLogger log;
+
+    /** */
+    private final String futPath;
+
+    /** */
+    private final Set<Long> remainingNodes;
+
+    /** */
+    private final Callable<Void> lsnr;
+
+    /**
+     * @param impl Disovery impl
+     * @param rtState Runtime state.
+     * @param futPath Future path.
+     * @param lsnr Future listener.
+     * @throws Exception If listener call failed.
+     */
+    ZkDistributedCollectDataFuture(
+        ZookeeperDiscoveryImpl impl,
+        ZkRuntimeState rtState,
+        String futPath,
+        Callable<Void> lsnr)
+        throws Exception
+    {
+        this.log = impl.log();
+        this.futPath = futPath;
+        this.lsnr = lsnr;
+
+        ZkClusterNodes top = rtState.top;
+
+        // Assume new nodes can not join while future is in progress.
+
+        remainingNodes = U.newHashSet(top.nodesByOrder.size());
+
+        for (ZookeeperClusterNode node : top.nodesByInternalId.values())
+            remainingNodes.add(node.order());
+
+        NodeResultsWatcher watcher = new NodeResultsWatcher(rtState, impl);
+
+        if (remainingNodes.isEmpty())
+            completeAndNotifyListener();
+        else {
+            if (log.isInfoEnabled()) {
+                log.info("Initialize data collect future [futPath=" + futPath + ", " +
+                    "remainingNodes=" + remainingNodes.size() + ']');
+            }
+
+            rtState.zkClient.getChildrenAsync(futPath, watcher, watcher);
+        }
+    }
+
+    /**
+     * @throws Exception If listener call failed.
+     */
+    private void completeAndNotifyListener() throws Exception {
+        if (super.onDone())
+            lsnr.call();
+    }
+
+    /**
+     * @param futPath
+     * @param client
+     * @param nodeOrder
+     * @param data
+     * @throws Exception If failed.
+     */
+    static void saveNodeResult(String futPath, ZookeeperClient client, long nodeOrder, byte[] data) throws Exception {
+        client.createIfNeeded(futPath + "/" + nodeOrder, data, CreateMode.PERSISTENT);
+    }
+
+    /**
+     * @param futPath
+     * @param client
+     * @param nodeOrder
+     * @return Node result data.
+     * @throws Exception If fai.ed
+     */
+    static byte[] readNodeResult(String futPath, ZookeeperClient client, long nodeOrder) throws Exception {
+        return client.getData(futPath + "/" + nodeOrder);
+    }
+
+    /**
+     * @param futResPath Result path.
+     * @param client Client.
+     * @param data Result data.
+     * @throws Exception If failed.
+     */
+    static void saveResult(String futResPath, ZookeeperClient client, byte[] data) throws Exception {
+        client.createIfNeeded(futResPath, data, CreateMode.PERSISTENT);
+    }
+
+    static byte[] readResult(ZookeeperClient client, ZkIgnitePaths paths, UUID futId) throws Exception {
+        return client.getData(paths.distributedFutureResultPath(futId));
+    }
+
+    /**
+     * @param client Client.
+     * @param paths Paths utils.
+     * @param futId Future ID.
+     * @param log Ignite Logger.
+     * @throws Exception If failed.
+     */
+    static void deleteFutureData(ZookeeperClient client,
+        ZkIgnitePaths paths,
+        UUID futId,
+        IgniteLogger log
+    ) throws Exception {
+        // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8189
+        String evtDir = paths.distributedFutureBasePath(futId);
+
+        try {
+            client.deleteAll(evtDir,
+                client.getChildren(evtDir),
+                -1);
+        }
+        catch (KeeperException.NoNodeException e) {
+            U.log(log, "Node for deletion was not found: " + e.getPath());
+
+            // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8189
+        }
+
+        client.deleteIfExists(evtDir, -1);
+
+        client.deleteIfExists(paths.distributedFutureResultPath(futId), -1);
+    }
+
+    /**
+     * @param top Current topology.
+     * @throws Exception If listener call failed.
+     */
+    void onTopologyChange(ZkClusterNodes top) throws Exception {
+        if (remainingNodes.isEmpty())
+            return;
+
+        for (Iterator<Long> it = remainingNodes.iterator(); it.hasNext();) {
+            Long nodeOrder = it.next();
+
+            if (!top.nodesByOrder.containsKey(nodeOrder)) {
+                it.remove();
+
+                int remaining = remainingNodes.size();
+
+                if (log.isInfoEnabled()) {
+                    log.info("ZkDistributedCollectDataFuture removed remaining failed node [node=" + nodeOrder +
+                        ", remaining=" + remaining +
+                        ", futPath=" + futPath + ']');
+                }
+
+                if (remaining == 0) {
+                    completeAndNotifyListener();
+
+                    break;
+                }
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    class NodeResultsWatcher extends ZkAbstractWatcher implements AsyncCallback.Children2Callback {
+        /**
+         * @param rtState Runtime state.
+         * @param impl Discovery impl.
+         */
+        NodeResultsWatcher(ZkRuntimeState rtState, ZookeeperDiscoveryImpl impl) {
+            super(rtState, impl);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void process0(WatchedEvent evt) {
+            if (evt.getType() == Watcher.Event.EventType.NodeChildrenChanged)
+                rtState.zkClient.getChildrenAsync(evt.getPath(), this, this);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
+            if (!onProcessStart())
+                return;
+
+            try {
+                if (!isDone()) {
+                    assert rc == 0 : KeeperException.Code.get(rc);
+
+                    for (int i = 0; i < children.size(); i++) {
+                        Long nodeOrder = Long.parseLong(children.get(i));
+
+                        if (remainingNodes.remove(nodeOrder)) {
+                            int remaining = remainingNodes.size();
+
+                            if (log.isInfoEnabled()) {
+                                log.info("ZkDistributedCollectDataFuture added new result [node=" + nodeOrder +
+                                    ", remaining=" + remaining +
+                                    ", futPath=" + path + ']');
+                            }
+
+                            if (remaining == 0)
+                                completeAndNotifyListener();
+                        }
+                    }
+                }
+
+                onProcessEnd();
+            }
+            catch (Throwable e) {
+                onProcessError(e);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkForceNodeFailMessage.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkForceNodeFailMessage.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkForceNodeFailMessage.java
new file mode 100644
index 0000000..de7291c
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkForceNodeFailMessage.java
@@ -0,0 +1,65 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Zk Force Node Fail Message.
+ */
+public class ZkForceNodeFailMessage implements DiscoverySpiCustomMessage, ZkInternalMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    final long nodeInternalId;
+
+    /** */
+    final String warning;
+
+    /**
+     * @param nodeInternalId Node ID.
+     * @param warning Warning to be displayed on all nodes.
+     */
+    ZkForceNodeFailMessage(long nodeInternalId, String warning) {
+        this.nodeInternalId = nodeInternalId;
+        this.warning = warning;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public DiscoverySpiCustomMessage ackMessage() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isMutable() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(ZkForceNodeFailMessage.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkIgnitePaths.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkIgnitePaths.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkIgnitePaths.java
new file mode 100644
index 0000000..9caf00f
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkIgnitePaths.java
@@ -0,0 +1,307 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.UUID;
+
+/**
+ *
+ */
+class ZkIgnitePaths {
+    /** */
+    static final String PATH_SEPARATOR = "/";
+
+    /** */
+    private static final byte CLIENT_NODE_FLAG_MASK = 0x01;
+
+    /** */
+    private static final int UUID_LEN = 36;
+
+    /** Directory to store joined node data. */
+    private static final String JOIN_DATA_DIR = "jd";
+
+    /** Directory to store new custom events. */
+    private static final String CUSTOM_EVTS_DIR = "ce";
+
+    /** Directory to store parts of multi-parts custom events. */
+    private static final String CUSTOM_EVTS_PARTS_DIR = "cp";
+
+    /** Directory to store acknowledge messages for custom events. */
+    private static final String CUSTOM_EVTS_ACKS_DIR = "ca";
+
+    /** Directory to store EPHEMERAL znodes for alive cluster nodes. */
+    static final String ALIVE_NODES_DIR = "n";
+
+    /** Path to store discovery events {@link ZkDiscoveryEventsData}. */
+    private static final String DISCO_EVENTS_PATH = "e";
+
+    /** */
+    final String clusterDir;
+
+    /** */
+    final String aliveNodesDir;
+
+    /** */
+    final String joinDataDir;
+
+    /** */
+    final String evtsPath;
+
+    /** */
+    final String customEvtsDir;
+
+    /** */
+    final String customEvtsPartsDir;
+
+    /** */
+    final String customEvtsAcksDir;
+
+    /**
+     * @param zkRootPath Base Zookeeper directory for all Ignite nodes.
+     */
+    ZkIgnitePaths(String zkRootPath) {
+        clusterDir = zkRootPath;
+
+        aliveNodesDir = zkPath(ALIVE_NODES_DIR);
+        joinDataDir = zkPath(JOIN_DATA_DIR);
+        evtsPath = zkPath(DISCO_EVENTS_PATH);
+        customEvtsDir = zkPath(CUSTOM_EVTS_DIR);
+        customEvtsPartsDir = zkPath(CUSTOM_EVTS_PARTS_DIR);
+        customEvtsAcksDir = zkPath(CUSTOM_EVTS_ACKS_DIR);
+    }
+
+    /**
+     * @param path Relative path.
+     * @return Full path.
+     */
+    private String zkPath(String path) {
+        return clusterDir + "/" + path;
+    }
+
+    /**
+     * @param nodeId Node ID.
+     * @param prefixId Unique prefix ID.
+     * @return Path.
+     */
+    String joiningNodeDataPath(UUID nodeId, UUID prefixId) {
+        return joinDataDir + '/' + prefixId + ":" + nodeId.toString();
+    }
+
+    /**
+     * @param path Alive node zk path.
+     * @return Node internal ID.
+     */
+    static long aliveInternalId(String path) {
+        int idx = path.lastIndexOf('|');
+
+        return Integer.parseInt(path.substring(idx + 1));
+    }
+
+    /**
+     * @param prefix Node unique path prefix.
+     * @param node Node.
+     * @return Path.
+     */
+    String aliveNodePathForCreate(String prefix, ZookeeperClusterNode node) {
+        byte flags = 0;
+
+        if (node.isClient())
+            flags |= CLIENT_NODE_FLAG_MASK;
+
+        return aliveNodesDir + "/" + prefix + ":" + node.id() + ":" + encodeFlags(flags) + "|";
+    }
+
+    /**
+     * @param path Alive node zk path.
+     * @return {@code True} if node is client.
+     */
+    static boolean aliveNodeClientFlag(String path) {
+        return (aliveFlags(path) & CLIENT_NODE_FLAG_MASK) != 0;
+    }
+
+    /**
+     * @param path Alive node zk path.
+     * @return Node ID.
+     */
+    static UUID aliveNodePrefixId(String path) {
+        return UUID.fromString(path.substring(0, ZkIgnitePaths.UUID_LEN));
+    }
+
+    /**
+     * @param path Alive node zk path.
+     * @return Node ID.
+     */
+    static UUID aliveNodeId(String path) {
+        // <uuid prefix>:<node id>:<flags>|<alive seq>
+        int startIdx = ZkIgnitePaths.UUID_LEN + 1;
+
+        String idStr = path.substring(startIdx, startIdx + ZkIgnitePaths.UUID_LEN);
+
+        return UUID.fromString(idStr);
+    }
+
+    /**
+     * @param path Event zk path.
+     * @return Event sequence number.
+     */
+    static int customEventSequence(String path) {
+        int idx = path.lastIndexOf('|');
+
+        return Integer.parseInt(path.substring(idx + 1));
+    }
+
+    /**
+     * @param path Custom event zl path.
+     * @return Event node ID.
+     */
+    static UUID customEventSendNodeId(String path) {
+        // <uuid prefix>:<node id>:<partCnt>|<seq>
+        int startIdx = ZkIgnitePaths.UUID_LEN + 1;
+
+        String idStr = path.substring(startIdx, startIdx + ZkIgnitePaths.UUID_LEN);
+
+        return UUID.fromString(idStr);
+    }
+
+    /**
+     * @param path Event path.
+     * @return Event unique prefix.
+     */
+    static String customEventPrefix(String path) {
+        // <uuid prefix>:<node id>:<partCnt>|<seq>
+
+        return path.substring(0, ZkIgnitePaths.UUID_LEN);
+    }
+
+    /**
+     * @param path Custom event zl path.
+     * @return Event node ID.
+     */
+    static int customEventPartsCount(String path) {
+        // <uuid prefix>:<node id>:<partCnt>|<seq>
+        int startIdx = 2 * ZkIgnitePaths.UUID_LEN + 2;
+
+        String cntStr = path.substring(startIdx, startIdx + 4);
+
+        int partCnt = Integer.parseInt(cntStr);
+
+        assert partCnt >= 1 : partCnt;
+
+        return partCnt;
+    }
+
+    /**
+     * @param prefix Prefix.
+     * @param nodeId Node ID.
+     * @param partCnt Parts count.
+     * @return Path.
+     */
+    String createCustomEventPath(String prefix, UUID nodeId, int partCnt) {
+        return customEvtsDir + "/" + prefix + ":" + nodeId + ":" + String.format("%04d", partCnt) + '|';
+    }
+
+    /**
+     * @param prefix Prefix.
+     * @param nodeId Node ID.
+     * @return Path.
+     */
+    String customEventPartsBasePath(String prefix, UUID nodeId) {
+        return customEvtsPartsDir + "/" + prefix + ":" + nodeId + ":";
+    }
+
+    /**
+     * @param prefix Prefix.
+     * @param nodeId Node ID.
+     * @param part Part number.
+     * @return Path.
+     */
+    String customEventPartPath(String prefix, UUID nodeId, int part) {
+        return customEventPartsBasePath(prefix, nodeId) + String.format("%04d", part);
+    }
+
+    /**
+     * @param evtId Event ID.
+     * @return Event zk path.
+     */
+    String joinEventDataPathForJoined(long evtId) {
+        return evtsPath + "/fj-" + evtId;
+    }
+
+    /**
+     * @param topVer Event topology version.
+     * @return Event zk path.
+     */
+    String joinEventSecuritySubjectPath(long topVer) {
+        return evtsPath + "/s-" + topVer;
+    }
+
+    /**
+     * @param origEvtId ID of original custom event.
+     * @return Path for custom event ack.
+     */
+    String ackEventDataPath(long origEvtId) {
+        assert origEvtId != 0;
+
+        return customEvtsAcksDir + "/" + String.valueOf(origEvtId);
+    }
+
+    /**
+     * @param id Future ID.
+     * @return Future path.
+     */
+    String distributedFutureBasePath(UUID id) {
+        return evtsPath + "/f-" + id;
+    }
+
+    /**
+     * @param id Future ID.
+     * @return Future path.
+     */
+    String distributedFutureResultPath(UUID id) {
+        return evtsPath + "/fr-" + id;
+    }
+
+    /**
+     * @param flags Flags.
+     * @return Flags string.
+     */
+    private static String encodeFlags(byte flags) {
+        int intVal = flags + 128;
+
+        String str = Integer.toString(intVal, 16);
+
+        if (str.length() == 1)
+            str = '0' + str;
+
+        assert str.length() == 2  : str;
+
+        return str;
+    }
+
+    /**
+     * @param path Alive node zk path.
+     * @return Flags.
+     */
+    private static byte aliveFlags(String path) {
+        int startIdx = path.lastIndexOf(':') + 1;
+
+        String flagsStr = path.substring(startIdx, startIdx + 2);
+
+        return (byte)(Integer.parseInt(flagsStr, 16) - 128);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalJoinErrorMessage.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalJoinErrorMessage.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalJoinErrorMessage.java
new file mode 100644
index 0000000..a73312c
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalJoinErrorMessage.java
@@ -0,0 +1,44 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+/**
+ *
+ */
+class ZkInternalJoinErrorMessage implements ZkInternalMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    transient boolean notifyNode = true;
+
+    /** */
+    final long nodeInternalId;
+
+    /** */
+    final String err;
+
+    /**
+     * @param nodeInternalId Joining node internal ID.
+     * @param err Error message.
+     */
+    ZkInternalJoinErrorMessage(long nodeInternalId, String err) {
+        this.nodeInternalId = nodeInternalId;
+        this.err = err;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalMessage.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalMessage.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalMessage.java
new file mode 100644
index 0000000..c1d56f0
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalMessage.java
@@ -0,0 +1,27 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+
+/**
+ *
+ */
+interface ZkInternalMessage extends Serializable {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinEventDataForJoined.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinEventDataForJoined.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinEventDataForJoined.java
new file mode 100644
index 0000000..e4ae4ba0
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinEventDataForJoined.java
@@ -0,0 +1,83 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+class ZkJoinEventDataForJoined implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final List<ZookeeperClusterNode> top;
+
+    /** */
+    private final Map<Long, byte[]> discoData;
+
+    /** */
+    private final Map<Long, Long> dupDiscoData;
+
+    /**
+     * @param top Topology.
+     * @param discoData Discovery data.
+     */
+    ZkJoinEventDataForJoined(List<ZookeeperClusterNode> top, Map<Long, byte[]> discoData, @Nullable Map<Long, Long> dupDiscoData) {
+        assert top != null;
+        assert discoData != null && !discoData.isEmpty();
+
+        this.top = top;
+        this.discoData = discoData;
+        this.dupDiscoData = dupDiscoData;
+    }
+
+    byte[] discoveryDataForNode(long nodeOrder) {
+        assert discoData != null;
+
+        byte[] dataBytes = discoData.get(nodeOrder);
+
+        if (dataBytes != null)
+            return dataBytes;
+
+        assert dupDiscoData != null;
+
+        Long dupDataNode = dupDiscoData.get(nodeOrder);
+
+        assert dupDataNode != null;
+
+        dataBytes = discoData.get(dupDataNode);
+
+        assert dataBytes != null;
+
+        return dataBytes;
+    }
+
+    /**
+     * @return Current topology.
+     */
+    List<ZookeeperClusterNode> topology() {
+        assert top != null;
+
+        return top;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinedNodeEvtData.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinedNodeEvtData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinedNodeEvtData.java
new file mode 100644
index 0000000..3c367cf
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinedNodeEvtData.java
@@ -0,0 +1,79 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import java.util.UUID;
+
+/**
+ * Zk Joined Node Evt Data.
+ */
+public class ZkJoinedNodeEvtData implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    final long topVer;
+
+    /** */
+    final long joinedInternalId;
+
+    /** */
+    final UUID nodeId;
+
+    /** */
+    final int joinDataPartCnt;
+
+    /** */
+    final int secSubjPartCnt;
+
+    /** */
+    final UUID joinDataPrefixId;
+
+    /** */
+    transient ZkJoiningNodeData joiningNodeData;
+
+    /**
+     * @param topVer Topology version for node join event.
+     * @param nodeId Joined node ID.
+     * @param joinedInternalId Joined node internal ID.
+     * @param joinDataPrefixId Join data unique prefix.
+     * @param joinDataPartCnt Join data part count.
+     * @param secSubjPartCnt Security subject part count.
+     */
+    ZkJoinedNodeEvtData(
+        long topVer,
+        UUID nodeId,
+        long joinedInternalId,
+        UUID joinDataPrefixId,
+        int joinDataPartCnt,
+        int secSubjPartCnt)
+    {
+        this.topVer = topVer;
+        this.nodeId = nodeId;
+        this.joinedInternalId = joinedInternalId;
+        this.joinDataPrefixId = joinDataPrefixId;
+        this.joinDataPartCnt = joinDataPartCnt;
+        this.secSubjPartCnt = secSubjPartCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "ZkJoinedNodeData [id=" + nodeId + ", order=" + topVer + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoiningNodeData.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoiningNodeData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoiningNodeData.java
new file mode 100644
index 0000000..ff8311d
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoiningNodeData.java
@@ -0,0 +1,87 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import java.util.Map;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+class ZkJoiningNodeData implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private int partCnt;
+
+    /** */
+    @GridToStringInclude
+    private ZookeeperClusterNode node;
+
+    /** */
+    @GridToStringInclude
+    private Map<Integer, Serializable> discoData;
+
+    /**
+     * @param partCnt Number of parts in multi-parts message.
+     */
+    ZkJoiningNodeData(int partCnt) {
+        this.partCnt = partCnt;
+    }
+
+    /**
+     * @param node Node.
+     * @param discoData Discovery data.
+     */
+    ZkJoiningNodeData(ZookeeperClusterNode node, Map<Integer, Serializable> discoData) {
+        assert node != null && node.id() != null : node;
+        assert discoData != null;
+
+        this.node = node;
+        this.discoData = discoData;
+    }
+
+    /**
+     * @return Number of parts in multi-parts message.
+     */
+    int partCount() {
+        return partCnt;
+    }
+
+    /**
+     * @return Node.
+     */
+    ZookeeperClusterNode node() {
+        return node;
+    }
+
+    /**
+     * @return Discovery data.
+     */
+    Map<Integer, Serializable> discoveryData() {
+        return discoData;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(ZkJoiningNodeData.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNoServersMessage.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNoServersMessage.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNoServersMessage.java
new file mode 100644
index 0000000..626fe74
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNoServersMessage.java
@@ -0,0 +1,50 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+class ZkNoServersMessage implements DiscoverySpiCustomMessage, ZkInternalMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Nullable @Override public DiscoverySpiCustomMessage ackMessage() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isMutable() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(ZkNoServersMessage.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNodeValidateResult.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNodeValidateResult.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNodeValidateResult.java
new file mode 100644
index 0000000..2abfee3
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNodeValidateResult.java
@@ -0,0 +1,43 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+/**
+ *
+ */
+class ZkNodeValidateResult {
+    /** */
+    String err;
+
+    /** */
+    byte[] secSubjZipBytes;
+
+    /**
+     * @param err Error.
+     */
+    ZkNodeValidateResult(String err) {
+        this.err = err;
+    }
+
+    /**
+     * @param secSubjZipBytes Marshalled security subject.
+     */
+    ZkNodeValidateResult(byte[] secSubjZipBytes) {
+        this.secSubjZipBytes = secSubjZipBytes;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRunnable.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRunnable.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRunnable.java
new file mode 100644
index 0000000..965bdc0
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRunnable.java
@@ -0,0 +1,51 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+/**
+ * Zk Runnable.
+ */
+public abstract class ZkRunnable extends ZkAbstractCallabck implements Runnable {
+    /**
+     * @param rtState Runtime state.
+     * @param impl Discovery impl.
+     */
+    ZkRunnable(ZkRuntimeState rtState, ZookeeperDiscoveryImpl impl) {
+        super(rtState, impl);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void run() {
+        if (!onProcessStart())
+            return;
+
+        try {
+            run0();
+
+            onProcessEnd();
+        }
+        catch (Throwable e) {
+            onProcessError(e);
+        }
+    }
+
+    /**
+     *
+     */
+    protected abstract void run0() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRuntimeState.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRuntimeState.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRuntimeState.java
new file mode 100644
index 0000000..cb04ac3
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRuntimeState.java
@@ -0,0 +1,135 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.List;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.spi.IgniteSpiTimeoutObject;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.Watcher;
+
+/**
+ *
+ */
+class ZkRuntimeState {
+    /** */
+    ZkWatcher watcher;
+
+    /** */
+    ZkAliveNodeDataWatcher aliveNodeDataWatcher;
+
+    /** */
+    volatile Exception errForClose;
+
+    /** */
+    final boolean prevJoined;
+
+    /** */
+    ZookeeperClient zkClient;
+
+    /** */
+    long internalOrder;
+
+    /** */
+    int joinDataPartCnt;
+
+    /** */
+    long gridStartTime;
+
+    /** */
+    volatile boolean joined;
+
+    /** */
+    ZkDiscoveryEventsData evtsData;
+
+    /** */
+    boolean crd;
+
+    /** */
+    String locNodeZkPath;
+
+    /** */
+    final ZkAliveNodeData locNodeInfo = new ZkAliveNodeData();
+
+    /** */
+    int procEvtCnt;
+
+    /** */
+    final ZkClusterNodes top = new ZkClusterNodes();
+
+    /** */
+    List<ClusterNode> commErrProcNodes;
+
+    /** Timeout callback registering watcher for join error
+     * (set this watcher after timeout as a minor optimization).
+     */
+    ZkTimeoutObject joinErrTo;
+
+    /** Timeout callback set to wait for join timeout. */
+    ZkTimeoutObject joinTo;
+
+    /** Timeout callback to update processed events counter. */
+    ZkTimeoutObject procEvtsUpdateTo;
+
+    /** */
+    boolean updateAlives;
+
+    /**
+     * @param prevJoined {@code True} if joined topology before reconnect attempt.
+     */
+    ZkRuntimeState(boolean prevJoined) {
+        this.prevJoined = prevJoined;
+    }
+
+    /**
+     * @param watcher Watcher.
+     * @param aliveNodeDataWatcher Alive nodes data watcher.
+     */
+    void init(ZkWatcher watcher, ZkAliveNodeDataWatcher aliveNodeDataWatcher) {
+        this.watcher = watcher;
+        this.aliveNodeDataWatcher = aliveNodeDataWatcher;
+    }
+
+    /**
+     * @param err Error.
+     */
+    void onCloseStart(Exception err) {
+        assert err != null;
+
+        errForClose = err;
+
+        ZookeeperClient zkClient = this.zkClient;
+
+        if (zkClient != null)
+            zkClient.onCloseStart();
+    }
+
+    /**
+     *
+     */
+    interface ZkWatcher extends Watcher, AsyncCallback.Children2Callback, AsyncCallback.DataCallback {
+        // No-op.
+    }
+
+    /**
+     *
+     */
+    interface ZkAliveNodeDataWatcher extends Watcher, AsyncCallback.DataCallback {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkTimeoutObject.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkTimeoutObject.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkTimeoutObject.java
new file mode 100644
index 0000000..4d3d5b4
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkTimeoutObject.java
@@ -0,0 +1,54 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.spi.IgniteSpiTimeoutObject;
+
+/**
+ *
+ */
+abstract class ZkTimeoutObject implements IgniteSpiTimeoutObject {
+    /** */
+    private final IgniteUuid id = IgniteUuid.randomUuid();
+
+    /** */
+    private final long endTime;
+
+    /** */
+    volatile boolean cancelled;
+
+    /**
+     * @param timeout Timeout.
+     */
+    ZkTimeoutObject(long timeout) {
+        long endTime = timeout >= 0 ? System.currentTimeMillis() + timeout : Long.MAX_VALUE;
+
+        this.endTime = endTime >= 0 ? endTime : Long.MAX_VALUE;
+    }
+
+    /** {@inheritDoc} */
+    @Override public final IgniteUuid id() {
+        return id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public final long endTime() {
+        return endTime;
+    }
+}


[07/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
index 54b3a78..e89a4c8 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
@@ -17,11 +17,21 @@
 
 package org.apache.ignite.spi.communication.tcp;
 
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
 import java.util.UUID;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CyclicBarrier;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.util.nio.GridCommunicationClient;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.spi.IgniteSpiAdapter;
 import org.apache.ignite.spi.communication.CommunicationSpi;
 import org.apache.ignite.spi.communication.GridAbstractCommunicationSelfTest;
@@ -85,6 +95,67 @@ abstract class GridTcpCommunicationSpiAbstractTest extends GridAbstractCommunica
         }
     }
 
+    /**
+     *
+     */
+    public void testCheckConnection1() {
+        for (int i = 0; i < 100; i++) {
+            for (Map.Entry<UUID, CommunicationSpi<Message>> entry : spis.entrySet()) {
+                TcpCommunicationSpi spi = (TcpCommunicationSpi)entry.getValue();
+
+                List<ClusterNode> checkNodes = new ArrayList<>(nodes);
+
+                assert checkNodes.size() > 1;
+
+                IgniteFuture<BitSet> fut = spi.checkConnection(checkNodes);
+
+                BitSet res = fut.get();
+
+                for (int n = 0; n < checkNodes.size(); n++)
+                    assertTrue(res.get(n));
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCheckConnection2() throws Exception {
+        final int THREADS = spis.size();
+
+        final CyclicBarrier b = new CyclicBarrier(THREADS);
+
+        List<IgniteInternalFuture> futs = new ArrayList<>();
+
+        for (Map.Entry<UUID, CommunicationSpi<Message>> entry : spis.entrySet()) {
+            final TcpCommunicationSpi spi = (TcpCommunicationSpi)entry.getValue();
+
+            futs.add(GridTestUtils.runAsync(new Callable() {
+                @Override public Object call() throws Exception {
+                    List<ClusterNode> checkNodes = new ArrayList<>(nodes);
+
+                    assert checkNodes.size() > 1;
+
+                    b.await();
+
+                    for (int i = 0; i < 100; i++) {
+                        IgniteFuture<BitSet> fut = spi.checkConnection(checkNodes);
+
+                        BitSet res = fut.get();
+
+                        for (int n = 0; n < checkNodes.size(); n++)
+                            assertTrue(res.get(n));
+                    }
+
+                    return null;
+                }
+            }));
+        }
+
+        for (IgniteInternalFuture f : futs)
+            f.get();
+    }
+
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         super.afterTest();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/spi/discovery/FilterDataForClientNodeDiscoveryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/FilterDataForClientNodeDiscoveryTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/FilterDataForClientNodeDiscoveryTest.java
index 54b48e5..9a45d2d 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/FilterDataForClientNodeDiscoveryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/FilterDataForClientNodeDiscoveryTest.java
@@ -206,6 +206,11 @@ public class FilterDataForClientNodeDiscoveryTest extends GridCommonAbstractTest
         }
 
         /** {@inheritDoc} */
+        @Override public boolean stopProcess() {
+            return false;
+        }
+
+        /** {@inheritDoc} */
         @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer,
             DiscoCache discoCache) {
             return null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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 ca05288..51dcb23 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
@@ -613,6 +613,16 @@ public class GridSpiTestContext implements IgniteSpiContext {
         return Collections.emptyMap();
     }
 
+    /** {@inheritDoc} */
+    @Override public boolean communicationFailureResolveSupported() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void resolveCommunicationFailure(ClusterNode node, Exception err) {
+        throw new UnsupportedOperationException();
+    }
+
     /**
      * @param cacheName Cache name.
      * @return Map representing cache.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java b/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java
index 4507572..e2594ca 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java
@@ -83,6 +83,15 @@ public final class GridTestProperties {
     /** "True value" enables {@link BinaryBasicNameMapper} in {@link BinaryTypeConfiguration#getNameMapper()}  */
     public static final String BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER = "binary.marshaller.use.simple.name.mapper";
 
+    /**
+     * Name of class which provides static method preprocessConfiguration(IgniteConfiguration cfg) to
+     * alter {@link org.apache.ignite.configuration.IgniteConfiguration} before node is started.
+     * <p>
+     * Note: this pre-preprocessor is started only if test starts node using one of GridAbstractTest's startGrid
+     * method.
+     */
+    public static final String IGNITE_CFG_PREPROCESSOR_CLS = "ignite.cfg.preprocessor.class";
+
     /** */
     static {
         // Initialize IGNITE_HOME system property.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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 c3b262c..f5784eb 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
@@ -46,6 +46,7 @@ import junit.framework.TestCase;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteClientDisconnectedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
@@ -84,6 +85,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgniteClosure;
+import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.logger.NullLogger;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
@@ -92,6 +94,7 @@ import org.apache.ignite.marshaller.jdk.JdkMarshaller;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointSpi;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
 import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.TestTcpDiscoverySpi;
@@ -124,6 +127,7 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.internal.GridKernalState.DISCONNECTED;
 import static org.apache.ignite.testframework.config.GridTestProperties.BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER;
+import static org.apache.ignite.testframework.config.GridTestProperties.IGNITE_CFG_PREPROCESSOR_CLS;
 
 /**
  * Common abstract test for Ignite tests.
@@ -203,13 +207,15 @@ public abstract class GridAbstractTest extends TestCase {
         if (BINARY_MARSHALLER)
             GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName());
 
-        Thread timer = new Thread(new GridTestClockTimer(), "ignite-clock-for-tests");
+        if (GridTestClockTimer.startTestTimer()) {
+            Thread timer = new Thread(new GridTestClockTimer(), "ignite-clock-for-tests");
 
-        timer.setDaemon(true);
+            timer.setDaemon(true);
 
-        timer.setPriority(10);
+            timer.setPriority(10);
 
-        timer.start();
+            timer.start();
+        }
     }
 
     /** */
@@ -838,6 +844,7 @@ public abstract class GridAbstractTest extends TestCase {
     protected Ignite startGrid(String igniteInstanceName, GridSpringResourceContext ctx) throws Exception {
         return startGrid(igniteInstanceName, optimize(getConfiguration(igniteInstanceName)), ctx);
     }
+
     /**
      * Starts new grid with given name.
      *
@@ -852,12 +859,33 @@ public abstract class GridAbstractTest extends TestCase {
             startingIgniteInstanceName.set(igniteInstanceName);
 
             try {
+                String cfgProcClsName = System.getProperty(IGNITE_CFG_PREPROCESSOR_CLS);
+
+                if (cfgProcClsName != null) {
+                    try {
+                        Class<?> cfgProc = Class.forName(cfgProcClsName);
+
+                        Method method = cfgProc.getMethod("preprocessConfiguration", IgniteConfiguration.class);
+
+                        if (!Modifier.isStatic(method.getModifiers()))
+                            throw new Exception("Non-static pre-processor method in pre-processor class: " + cfgProcClsName);
+
+                        method.invoke(null, cfg);
+                    }
+                    catch (Exception e) {
+                        log.error("Failed to pre-process IgniteConfiguration using pre-processor class: " + cfgProcClsName);
+
+                        throw new IgniteException(e);
+                    }
+                }
+
                 Ignite node = IgnitionEx.start(cfg, ctx);
 
                 IgniteConfiguration nodeCfg = node.configuration();
 
                 log.info("Node started with the following configuration [id=" + node.cluster().localNode().id()
                     + ", marshaller=" + nodeCfg.getMarshaller()
+                    + ", discovery=" + nodeCfg.getDiscoverySpi()
                     + ", binaryCfg=" + nodeCfg.getBinaryConfiguration()
                     + ", lateAff=" + nodeCfg.isLateAffinityAssignment() + "]");
 
@@ -967,6 +995,26 @@ public abstract class GridAbstractTest extends TestCase {
         if (cfg == null)
             cfg = optimize(getConfiguration(igniteInstanceName));
 
+        if (locNode != null) {
+            DiscoverySpi discoverySpi = locNode.configuration().getDiscoverySpi();
+
+            if (discoverySpi != null && !(discoverySpi instanceof TcpDiscoverySpi)) {
+                try {
+                    // Clone added to support ZookeeperDiscoverySpi.
+                    Method m = discoverySpi.getClass().getDeclaredMethod("cloneSpiConfiguration");
+
+                    m.setAccessible(true);
+
+                    cfg.setDiscoverySpi((DiscoverySpi) m.invoke(discoverySpi));
+
+                    resetDiscovery = false;
+                }
+                catch (NoSuchMethodException e) {
+                    // Ignore.
+                }
+            }
+        }
+
         return new IgniteProcessProxy(cfg, log, locNode, resetDiscovery);
     }
 
@@ -1075,7 +1123,9 @@ public abstract class GridAbstractTest extends TestCase {
             for (Ignite g : srvs)
                 stopGrid(g.name(), cancel, false);
 
-            assert G.allGrids().isEmpty();
+            List<Ignite> nodes = G.allGrids();
+
+            assert nodes.isEmpty() : nodes;
         }
         finally {
             IgniteProcessProxy.killAll(); // In multi-JVM case.
@@ -1177,6 +1227,14 @@ public abstract class GridAbstractTest extends TestCase {
     }
 
     /**
+     * @param nodeIdx Node index.
+     * @return Node ID.
+     */
+    protected final UUID nodeId(int nodeIdx) {
+        return ignite(nodeIdx).cluster().localNode().id();
+    }
+
+    /**
      * Gets grid for given test.
      *
      * @return Grid for given test.
@@ -1217,7 +1275,11 @@ public abstract class GridAbstractTest extends TestCase {
      * @throws Exception If failed.
      */
     protected Ignite startGrid(String igniteInstanceName, String springCfgPath) throws Exception {
-        return startGrid(igniteInstanceName, loadConfiguration(springCfgPath));
+        IgniteConfiguration cfg = loadConfiguration(springCfgPath);
+
+        cfg.setGridLogger(getTestResources().getLogger());
+
+        return startGrid(igniteInstanceName, cfg);
     }
 
     /**
@@ -2142,6 +2204,50 @@ public abstract class GridAbstractTest extends TestCase {
             }
         }
     }
+    /**
+     * @param expSize Expected nodes number.
+     * @throws Exception If failed.
+     */
+    protected void waitForTopology(final int expSize) throws Exception {
+        assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                List<Ignite> nodes = G.allGrids();
+
+                if (nodes.size() != expSize) {
+                    info("Wait all nodes [size=" + nodes.size() + ", exp=" + expSize + ']');
+
+                    return false;
+                }
+
+                for (Ignite node: nodes) {
+                    try {
+                        IgniteFuture<?> reconnectFut = node.cluster().clientReconnectFuture();
+
+                        if (reconnectFut != null && !reconnectFut.isDone()) {
+                            info("Wait for size on node, reconnect is in progress [node=" + node.name() + ']');
+
+                            return false;
+                        }
+
+                        int sizeOnNode = node.cluster().nodes().size();
+
+                        if (sizeOnNode != expSize) {
+                            info("Wait for size on node [node=" + node.name() + ", size=" + sizeOnNode + ", exp=" + expSize + ']');
+
+                            return false;
+                        }
+                    }
+                    catch (IgniteClientDisconnectedException e) {
+                        info("Wait for size on node, node disconnected [node=" + node.name() + ']');
+
+                        return false;
+                    }
+                }
+
+                return true;
+            }
+        }, 30_000));
+    }
 
     /**
      * @param millis Time to sleep.
@@ -2172,6 +2278,17 @@ public abstract class GridAbstractTest extends TestCase {
     }
 
     /**
+     * @return {@code True} if nodes use {@link TcpDiscoverySpi}.
+     */
+    protected static boolean tcpDiscovery() {
+        List<Ignite> nodes = G.allGrids();
+
+        assertFalse("There are no nodes", nodes.isEmpty());
+
+        return nodes.get(0).configuration().getDiscoverySpi() instanceof TcpDiscoverySpi;
+    }
+
+    /**
      *
      */
     private static interface WriteReplaceOwner {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
index d7be576..2b3a19c 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
@@ -162,6 +162,8 @@ public class IgniteNodeRunner {
                 cfg.setDiscoverySpi(disco);
             }
 
+            X.println("Configured discovery: " + cfg.getDiscoverySpi().getClass().getName());
+
             return cfg;
         }
         finally {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
index 55fab8d..14eb296 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
@@ -19,6 +19,7 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.ClusterNodeMetricsSelfTest;
+import org.apache.ignite.internal.ClusterNodeMetricsUpdateTest;
 import org.apache.ignite.internal.GridAffinityNoCacheSelfTest;
 import org.apache.ignite.internal.GridAffinitySelfTest;
 import org.apache.ignite.internal.GridAlwaysFailoverSpiFailSelfTest;
@@ -122,6 +123,7 @@ public class IgniteComputeGridTestSuite {
         suite.addTestSuite(GridAlwaysFailoverSpiFailSelfTest.class);
         suite.addTestSuite(GridTaskInstanceExecutionSelfTest.class);
         suite.addTestSuite(ClusterNodeMetricsSelfTest.class);
+        suite.addTestSuite(ClusterNodeMetricsUpdateTest.class);
         suite.addTestSuite(GridNonHistoryMetricsSelfTest.class);
         suite.addTestSuite(GridCancelledJobsMetricsSelfTest.class);
         suite.addTestSuite(GridCollisionJobsContextSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryCancelSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryCancelSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryCancelSelfTest.java
index b9ef1e4..e26b211 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryCancelSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryCancelSelfTest.java
@@ -117,7 +117,7 @@ public class IgniteCacheDistributedQueryCancelSelfTest extends GridCommonAbstrac
                     }
 
                     for (Ignite g : G.allGrids())
-                        if (!g.configuration().getDiscoverySpi().isClientMode())
+                        if (!g.configuration().isClientMode())
                             stopGrid(g.name(), true);
                 }
             }, 1);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
index 97720d5..bd3b093 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
@@ -89,11 +89,14 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
      * @param mode Mode.
      * @param atomicityMode Atomicity mode.
      * @param near Near flag.
+     * @throws Exception If failed.
      */
     private void initialize(CacheMode mode, CacheAtomicityMode atomicityMode, boolean near)
-        throws IgniteCheckedException {
+        throws Exception {
         createSqlCache(node(), cacheConfiguration(mode, atomicityMode, near));
 
+        awaitPartitionMapExchange();
+
         grid(IDX_CLI_NEAR_ONLY).getOrCreateNearCache(CACHE_NAME, new NearCacheConfiguration<>());
 
         assertNoIndex(CACHE_NAME, TBL_NAME, IDX_NAME_1);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridJtaTransactionManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridJtaTransactionManagerSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridJtaTransactionManagerSelfTest.java
index a181068..5cad167 100644
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridJtaTransactionManagerSelfTest.java
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridJtaTransactionManagerSelfTest.java
@@ -44,13 +44,7 @@ public class GridJtaTransactionManagerSelfTest extends GridCommonAbstractTest {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName).
             setCacheConfiguration(defaultCacheConfiguration().setCacheMode(PARTITIONED));
 
-        cfg.getTransactionConfiguration().setTxManagerFactory(new Factory<TransactionManager>() {
-            private static final long serialVersionUID = 0L;
-
-            @Override public TransactionManager create() {
-                return jotm.getTransactionManager();
-            }
-        });
+        cfg.getTransactionConfiguration().setTxManagerFactory(new TestTxManagerFactory());
 
         return cfg;
     }
@@ -205,4 +199,17 @@ public class GridJtaTransactionManagerSelfTest extends GridCommonAbstractTest {
             cache.removeAll();
         }
     }
+
+    /**
+     *
+     */
+    static class TestTxManagerFactory implements Factory<TransactionManager> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public TransactionManager create() {
+            return jotm.getTransactionManager();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactorySelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactorySelfTest.java
index f6fd5c7..14b7fae 100644
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactorySelfTest.java
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactorySelfTest.java
@@ -30,12 +30,19 @@ public class GridPartitionedCacheJtaFactorySelfTest extends AbstractCacheJtaSelf
     @Override protected void configureJta(IgniteConfiguration cfg) {
         TransactionConfiguration txCfg = cfg.getTransactionConfiguration();
 
-        txCfg.setTxManagerFactory(new Factory<TransactionManager>() {
-            private static final long serialVersionUID = 0L;
+        txCfg.setTxManagerFactory(new TestTxManagerFactory());
+    }
+
+    /**
+     *
+     */
+    static class TestTxManagerFactory implements Factory<TransactionManager> {
+        /** */
+        private static final long serialVersionUID = 0L;
 
-            @Override public TransactionManager create() {
-                return jotm.getTransactionManager();
-            }
-        });
+        /** {@inheritDoc} */
+        @Override public TransactionManager create() {
+            return jotm.getTransactionManager();
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
index fce47a6..d87ea0a 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.processors.cache.query.QueryCursorEx
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata
 import org.apache.ignite.lang.IgniteUuid
 import org.apache.ignite.spark.impl._
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi
 import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode
 import org.apache.spark._
 import org.apache.spark.rdd.RDD
@@ -91,8 +92,14 @@ class IgniteRDD[K, V] (
     override protected[spark] def getPreferredLocations(split: Partition): Seq[String] = {
         ensureCache()
 
-        ic.ignite().affinity(cacheName).mapPartitionToPrimaryAndBackups(split.index)
+        if (ic.ignite().configuration().getDiscoverySpi().isInstanceOf[TcpDiscoverySpi]) {
+          ic.ignite().affinity(cacheName).mapPartitionToPrimaryAndBackups(split.index)
             .map(_.asInstanceOf[TcpDiscoveryNode].socketAddresses()).flatten.map(_.getHostName).toList
+        }
+        else {
+          ic.ignite().affinity(cacheName).mapPartitionToPrimaryAndBackups(split.index)
+            .flatten(_.hostNames).toSeq
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java b/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java
index 75128fc..b453858 100644
--- a/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java
+++ b/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java
@@ -998,7 +998,8 @@ public class GridFactorySelfTest extends GridCommonAbstractTest {
 
             startGrid("1", c);
 
-            assert ((TcpDiscoverySpi)c.getDiscoverySpi()).started();
+            if (tcpDiscovery())
+                assert ((TcpDiscoverySpi)c.getDiscoverySpi()).started();
 
             try {
                 startGrid("2", c);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/spring/src/test/java/org/apache/ignite/p2p/GridP2PUserVersionChangeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/p2p/GridP2PUserVersionChangeSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/p2p/GridP2PUserVersionChangeSelfTest.java
index b861e19..46da3cc 100644
--- a/modules/spring/src/test/java/org/apache/ignite/p2p/GridP2PUserVersionChangeSelfTest.java
+++ b/modules/spring/src/test/java/org/apache/ignite/p2p/GridP2PUserVersionChangeSelfTest.java
@@ -44,6 +44,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.testsuites.IgniteIgnore;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
+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.events.EventType.EVT_TASK_UNDEPLOYED;
 
@@ -255,12 +256,12 @@ public class GridP2PUserVersionChangeSelfTest extends GridCommonAbstractTest {
 
             ignite2.events().localListen(new IgnitePredicate<Event>() {
                 @Override public boolean apply(Event evt) {
-                    if (evt.type() == EVT_NODE_LEFT)
+                    if (evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED)
                         discoLatch.countDown();
 
                     return true;
                 }
-            }, EVT_NODE_LEFT);
+            }, EVT_NODE_LEFT, EVT_NODE_FAILED);
 
             Integer res1 = (Integer)ignite1.compute().execute(task1, ignite2.cluster().localNode().id());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/yardstick/pom-standalone.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom-standalone.xml b/modules/yardstick/pom-standalone.xml
index 577a95e..6905d94 100644
--- a/modules/yardstick/pom-standalone.xml
+++ b/modules/yardstick/pom-standalone.xml
@@ -54,6 +54,12 @@
 
         <dependency>
             <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-zookeeper</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
             <artifactId>ignite-log4j</artifactId>
             <version>${project.version}</version>
         </dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index 8cad24b..9923bb7 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -55,6 +55,12 @@
 
         <dependency>
             <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-zookeeper</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
             <artifactId>ignite-log4j</artifactId>
             <version>${project.version}</version>
         </dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/modules/zookeeper/pom.xml b/modules/zookeeper/pom.xml
index c3c3679..2d47ece 100644
--- a/modules/zookeeper/pom.xml
+++ b/modules/zookeeper/pom.xml
@@ -49,6 +49,12 @@
 
         <dependency>
             <groupId>org.apache.curator</groupId>
+            <artifactId>curator-recipes</artifactId>
+            <version>${curator.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.curator</groupId>
             <artifactId>curator-x-discovery</artifactId>
             <version>${curator.version}</version>
         </dependency>
@@ -109,6 +115,13 @@
 
         <dependency>
             <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-indexing</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
             <artifactId>ignite-log4j</artifactId>
             <version>${project.version}</version>
             <scope>test</scope>
@@ -122,16 +135,43 @@
         </dependency>
 
         <dependency>
+            <groupId>com.thoughtworks.xstream</groupId>
+            <artifactId>xstream</artifactId>
+            <version>1.4.8</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
             <groupId>org.apache.ignite</groupId>
             <artifactId>ignite-core</artifactId>
             <version>${project.version}</version>
             <type>test-jar</type>
             <scope>test</scope>
         </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-indexing</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 
     <build>
         <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+
             <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
             <plugin>
                 <groupId>org.apache.felix</groupId>

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java
new file mode 100644
index 0000000..860c71c
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java
@@ -0,0 +1,557 @@
+/*
+ * 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.spi.discovery.zk;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.curator.utils.PathUtils;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpiInternalListener;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteProductVersion;
+import org.apache.ignite.resources.LoggerResource;
+import org.apache.ignite.spi.IgniteSpiAdapter;
+import org.apache.ignite.spi.IgniteSpiConfiguration;
+import org.apache.ignite.spi.IgniteSpiContext;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.IgniteSpiMultipleInstancesSupport;
+import org.apache.ignite.spi.communication.CommunicationSpi;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.DiscoveryMetricsProvider;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.apache.ignite.spi.discovery.DiscoverySpiDataExchange;
+import org.apache.ignite.spi.discovery.DiscoverySpiHistorySupport;
+import org.apache.ignite.spi.discovery.DiscoverySpiListener;
+import org.apache.ignite.spi.discovery.DiscoverySpiMutableCustomMessageSupport;
+import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator;
+import org.apache.ignite.spi.discovery.DiscoverySpiOrderSupport;
+import org.apache.ignite.spi.discovery.zk.internal.ZookeeperClusterNode;
+import org.apache.ignite.spi.discovery.zk.internal.ZookeeperDiscoveryImpl;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_CONSISTENT_ID_BY_HOST_WITHOUT_PORT;
+import static org.apache.ignite.IgniteSystemProperties.getBoolean;
+
+/**
+ * Zookeeper Discovery Spi.
+ */
+@IgniteSpiMultipleInstancesSupport(true)
+@DiscoverySpiOrderSupport(true)
+@DiscoverySpiHistorySupport(true)
+@DiscoverySpiMutableCustomMessageSupport(false)
+public class ZookeeperDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, IgniteDiscoverySpi {
+    /** */
+    public static final String DFLT_ROOT_PATH = "/apacheIgnite";
+
+    /** */
+    public static final long DFLT_JOIN_TIMEOUT = 0;
+
+    /** */
+    @GridToStringInclude
+    private String zkRootPath = DFLT_ROOT_PATH;
+
+    /** */
+    @GridToStringInclude
+    private String zkConnectionString;
+
+    /** */
+    private long joinTimeout = DFLT_JOIN_TIMEOUT;
+
+    /** */
+    @GridToStringInclude
+    private long sesTimeout;
+
+    /** */
+    private boolean clientReconnectDisabled;
+
+    /** */
+    @GridToStringExclude
+    private DiscoverySpiListener lsnr;
+
+    /** */
+    @GridToStringExclude
+    private DiscoverySpiDataExchange exchange;
+
+    /** */
+    @GridToStringExclude
+    private DiscoverySpiNodeAuthenticator nodeAuth;
+
+    /** */
+    @GridToStringExclude
+    private DiscoveryMetricsProvider metricsProvider;
+
+    /** */
+    @GridToStringExclude
+    private ZookeeperDiscoveryImpl impl;
+
+    /** */
+    @GridToStringExclude
+    private Map<String, Object> locNodeAttrs;
+
+    /** */
+    @GridToStringExclude
+    private IgniteProductVersion locNodeVer;
+
+    /** */
+    @GridToStringExclude
+    private Serializable consistentId;
+
+    /** Local node addresses. */
+    private IgniteBiTuple<Collection<String>, Collection<String>> addrs;
+
+    /** */
+    @LoggerResource
+    @GridToStringExclude
+    private IgniteLogger log;
+
+    /** */
+    private IgniteDiscoverySpiInternalListener internalLsnr;
+
+    /**
+     * @return Base path in ZK for znodes created by SPI.
+     */
+    public String getZkRootPath() {
+        return zkRootPath;
+    }
+
+    /**
+     * @param zkRootPath Base path in ZooKeeper for znodes created by SPI.
+     * @return {@code this} for chaining.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public ZookeeperDiscoverySpi setZkRootPath(String zkRootPath) {
+        this.zkRootPath = zkRootPath;
+
+        return this;
+    }
+
+    /**
+     * @return ZooKeeper session timeout.
+     */
+    public long getSessionTimeout() {
+        return sesTimeout;
+    }
+
+    /**
+     * @param sesTimeout ZooKeeper session timeout.
+     * @return {@code this} for chaining.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public ZookeeperDiscoverySpi setSessionTimeout(long sesTimeout) {
+        this.sesTimeout = sesTimeout;
+
+        return this;
+    }
+
+    /**
+     * @return Cluster join timeout.
+     */
+    public long getJoinTimeout() {
+        return joinTimeout;
+    }
+
+    /**
+     * @param joinTimeout Cluster join timeout ({@code 0} means wait forever).
+     * @return {@code this} for chaining.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public ZookeeperDiscoverySpi setJoinTimeout(long joinTimeout) {
+        this.joinTimeout = joinTimeout;
+
+        return this;
+    }
+
+    /**
+     * @return ZooKeeper connection string
+     */
+    public String getZkConnectionString() {
+        return zkConnectionString;
+    }
+
+    /**
+     * @param zkConnectionString ZooKeeper connection string
+     * @return {@code this} for chaining.
+     */
+    @IgniteSpiConfiguration(optional = false)
+    public ZookeeperDiscoverySpi setZkConnectionString(String zkConnectionString) {
+        this.zkConnectionString = zkConnectionString;
+
+        return this;
+    }
+
+    /**
+     * If {@code true} client does not try to reconnect.
+     *
+     * @return Client reconnect disabled flag.
+     */
+    public boolean isClientReconnectDisabled() {
+        return clientReconnectDisabled;
+    }
+
+    /**
+     * Sets client reconnect disabled flag.
+     *
+     * @param clientReconnectDisabled Client reconnect disabled flag.
+     * @return {@code this} for chaining.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public ZookeeperDiscoverySpi setClientReconnectDisabled(boolean clientReconnectDisabled) {
+        this.clientReconnectDisabled = clientReconnectDisabled;
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean clientReconnectSupported() {
+        return !clientReconnectDisabled;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clientReconnect() {
+        impl.reconnect();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean knownNode(UUID nodeId) {
+        return impl.knownNode(nodeId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsCommunicationFailureResolve() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void resolveCommunicationFailure(ClusterNode node, Exception err) {
+        impl.resolveCommunicationError(node, err);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Serializable consistentId() throws IgniteSpiException {
+        if (consistentId == null) {
+            consistentId = ignite.configuration().getConsistentId();
+
+            if (consistentId == null) {
+                initAddresses();
+
+                final List<String> sortedAddrs = new ArrayList<>(addrs.get1());
+
+                Collections.sort(sortedAddrs);
+
+                if (getBoolean(IGNITE_CONSISTENT_ID_BY_HOST_WITHOUT_PORT))
+                    consistentId = U.consistentId(sortedAddrs);
+                else {
+                    Integer commPort = null;
+
+                    if (locNodeAttrs != null) {
+                        commPort = (Integer)locNodeAttrs.get(
+                            TcpCommunicationSpi.class.getSimpleName() + "." + TcpCommunicationSpi.ATTR_PORT);
+                    }
+                    else {
+                        CommunicationSpi commSpi = ignite.configuration().getCommunicationSpi();
+
+                        if (commSpi instanceof TcpCommunicationSpi) {
+                            commPort = ((TcpCommunicationSpi)commSpi).boundPort();
+
+                            if (commPort == -1)
+                                commPort = null;
+                        }
+                    }
+
+                    if (commPort == null) {
+                        U.warn(log, "Can not initialize default consistentId, TcpCommunicationSpi port is not initialized.");
+
+                        consistentId = ignite.configuration().getNodeId();
+                    }
+                    else
+                        consistentId = U.consistentId(sortedAddrs, commPort);
+                }
+            }
+        }
+
+        return consistentId;
+    }
+
+    /**
+     *
+     */
+    private void initAddresses() {
+        if (addrs == null) {
+            String locHost = ignite != null ? ignite.configuration().getLocalHost() : null;
+
+            InetAddress locAddr;
+
+            try {
+                locAddr = U.resolveLocalHost(locHost);
+            }
+            catch (IOException e) {
+                throw new IgniteSpiException("Unknown local address: " + locHost, e);
+            }
+
+            try {
+                addrs = U.resolveLocalAddresses(locAddr);
+            }
+            catch (Exception e) {
+                throw new IgniteSpiException("Failed to resolve local host to set of external addresses: " + locHost,
+                    e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<ClusterNode> getRemoteNodes() {
+        return impl.remoteNodes();
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterNode getLocalNode() {
+        return impl != null ? impl.localNode() : null;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public ClusterNode getNode(UUID nodeId) {
+        return impl.node(nodeId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean pingNode(UUID nodeId) {
+        return impl.pingNode(nodeId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setNodeAttributes(Map<String, Object> attrs, IgniteProductVersion ver) {
+        assert locNodeAttrs == null;
+        assert locNodeVer == null;
+
+        if (log.isDebugEnabled()) {
+            log.debug("Node attributes to set: " + attrs);
+            log.debug("Node version to set: " + ver);
+        }
+
+        locNodeAttrs = attrs;
+        locNodeVer = ver;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setListener(@Nullable DiscoverySpiListener lsnr) {
+        this.lsnr = lsnr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setDataExchange(DiscoverySpiDataExchange exchange) {
+        this.exchange = exchange;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setMetricsProvider(DiscoveryMetricsProvider metricsProvider) {
+        this.metricsProvider = metricsProvider;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void disconnect() throws IgniteSpiException {
+        impl.stop();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setAuthenticator(DiscoverySpiNodeAuthenticator auth) {
+        this.nodeAuth = auth;
+    }
+
+    /**
+     * @return Authenticator.
+     */
+    public DiscoverySpiNodeAuthenticator getAuthenticator() {
+        return nodeAuth;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getGridStartTime() {
+        return impl.gridStartTime();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) {
+        IgniteDiscoverySpiInternalListener internalLsnr = impl.internalLsnr;
+
+        if (internalLsnr != null) {
+            if (!internalLsnr.beforeSendCustomEvent(this, log, msg))
+                return;
+        }
+
+        impl.sendCustomMessage(msg);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void failNode(UUID nodeId, @Nullable String warning) {
+        impl.failNode(nodeId, warning);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isClientMode() throws IllegalStateException {
+        return impl.localNode().isClient();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
+        super.onContextInitialized0(spiCtx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(@Nullable String igniteInstanceName) throws IgniteSpiException {
+        if (sesTimeout == 0)
+            sesTimeout = ignite.configuration().getFailureDetectionTimeout().intValue();
+
+        assertParameter(sesTimeout > 0, "sessionTimeout > 0");
+
+        A.notNullOrEmpty(zkConnectionString, "zkConnectionString can not be empty");
+
+        A.notNullOrEmpty(zkRootPath, "zkRootPath can not be empty");
+
+        zkRootPath = zkRootPath.trim();
+
+        if (zkRootPath.endsWith("/"))
+            zkRootPath = zkRootPath.substring(0, zkRootPath.length() - 1);
+
+        try {
+            PathUtils.validatePath(zkRootPath);
+        }
+        catch (IllegalArgumentException e) {
+            throw new IgniteSpiException("zkRootPath is invalid: " + zkRootPath, e);
+        }
+
+        ZookeeperClusterNode locNode = initLocalNode();
+
+        if (log.isInfoEnabled()) {
+            log.info("Start Zookeeper discovery [zkConnectionString=" + zkConnectionString +
+                ", sessionTimeout=" + sesTimeout +
+                ", zkRootPath=" + zkRootPath + ']');
+        }
+
+        impl = new ZookeeperDiscoveryImpl(
+            this,
+            igniteInstanceName,
+            log,
+            zkRootPath,
+            locNode,
+            lsnr,
+            exchange,
+            internalLsnr);
+
+        try {
+            impl.startJoinAndWait();
+        }
+        catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IgniteSpiException("Failed to join cluster, thread was interrupted", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setInternalListener(IgniteDiscoverySpiInternalListener lsnr) {
+        if (impl != null)
+            impl.internalLsnr = lsnr;
+        else
+            internalLsnr = lsnr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void simulateNodeFailure() {
+        impl.simulateNodeFailure();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStop() throws IgniteSpiException {
+        if (impl != null)
+            impl.stop();
+    }
+
+    /**
+     * @return Local node instance.
+     */
+    private ZookeeperClusterNode initLocalNode() {
+        assert ignite != null;
+
+        initAddresses();
+
+        ZookeeperClusterNode locNode = new ZookeeperClusterNode(
+            ignite.configuration().getNodeId(),
+            addrs.get1(),
+            addrs.get2(),
+            locNodeVer,
+            locNodeAttrs,
+            consistentId(),
+            sesTimeout,
+            ignite.configuration().isClientMode(),
+            metricsProvider);
+
+        locNode.local(true);
+
+        DiscoverySpiListener lsnr = this.lsnr;
+
+        if (lsnr != null)
+            lsnr.onLocalNodeInitialized(locNode);
+
+        if (log.isDebugEnabled())
+            log.debug("Local node initialized: " + locNode);
+
+        if (metricsProvider != null) {
+            locNode.setMetrics(metricsProvider.metrics());
+            locNode.setCacheMetrics(metricsProvider.cacheMetrics());
+        }
+
+        return locNode;
+    }
+
+    /**
+     * Used in tests (called via reflection).
+     *
+     * @return Copy of SPI.
+     */
+    private ZookeeperDiscoverySpi cloneSpiConfiguration() {
+        ZookeeperDiscoverySpi spi = new ZookeeperDiscoverySpi();
+
+        spi.setZkRootPath(zkRootPath);
+        spi.setZkConnectionString(zkConnectionString);
+        spi.setSessionTimeout(sesTimeout);
+        spi.setJoinTimeout(joinTimeout);
+        spi.setClientReconnectDisabled(clientReconnectDisabled);
+
+        return spi;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(ZookeeperDiscoverySpi.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractCallabck.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractCallabck.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractCallabck.java
new file mode 100644
index 0000000..b80a9dd
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractCallabck.java
@@ -0,0 +1,83 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+
+/**
+ *
+ */
+abstract class ZkAbstractCallabck {
+    /** */
+    final ZkRuntimeState rtState;
+
+    /** */
+    private final ZookeeperDiscoveryImpl impl;
+
+    /** */
+    private final GridSpinBusyLock busyLock;
+
+    /**
+     * @param rtState Runtime state.
+     * @param impl Discovery impl.
+     */
+    ZkAbstractCallabck(ZkRuntimeState rtState, ZookeeperDiscoveryImpl impl) {
+        this.rtState = rtState;
+        this.impl = impl;
+
+        busyLock = impl.busyLock;
+    }
+
+    /**
+     * @return {@code True} if is able to start processing.
+     */
+    final boolean onProcessStart() {
+        boolean start = rtState.errForClose == null && busyLock.enterBusy();
+
+        if (!start) {
+            assert rtState.errForClose != null;
+
+            onStartFailed();
+
+            return false;
+        }
+
+        return true;
+    }
+
+    /**
+     *
+     */
+    void onStartFailed() {
+        // No-op.
+    }
+
+    /**
+     *
+     */
+    final void onProcessEnd() {
+        busyLock.leaveBusy();
+    }
+
+    /**
+     * @param e Error.
+     */
+    final void onProcessError(Throwable e) {
+        impl.onFatalError(busyLock, e);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractChildrenCallback.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractChildrenCallback.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractChildrenCallback.java
new file mode 100644
index 0000000..2292e35
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractChildrenCallback.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.spi.discovery.zk.internal;
+
+import java.util.List;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ *
+ */
+abstract class ZkAbstractChildrenCallback extends ZkAbstractCallabck implements AsyncCallback.Children2Callback {
+    /**
+     * @param rtState Runtime state.
+     * @param impl Discovery impl.
+     */
+    ZkAbstractChildrenCallback(ZkRuntimeState rtState, ZookeeperDiscoveryImpl impl) {
+        super(rtState, impl);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
+        if (!onProcessStart())
+            return;
+
+        try {
+            processResult0(rc, path, ctx, children, stat);
+
+            onProcessEnd();
+        }
+        catch (Throwable e) {
+            onProcessError(e);
+        }
+    }
+
+    /**
+     * @param rc
+     * @param path
+     * @param ctx
+     * @param children
+     * @param stat
+     * @throws Exception If failed.
+     */
+    abstract void processResult0(int rc, String path, Object ctx, List<String> children, Stat stat)
+        throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractWatcher.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractWatcher.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractWatcher.java
new file mode 100644
index 0000000..9098d05
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractWatcher.java
@@ -0,0 +1,55 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+
+/**
+ *
+ */
+abstract class ZkAbstractWatcher extends ZkAbstractCallabck implements Watcher {
+    /**
+     * @param rtState Runtime state.
+     * @param impl Discovery impl.
+     */
+    ZkAbstractWatcher(ZkRuntimeState rtState, ZookeeperDiscoveryImpl impl) {
+        super(rtState, impl);
+    }
+
+    /** {@inheritDoc} */
+    @Override public final void process(WatchedEvent evt) {
+        if (!onProcessStart())
+            return;
+
+        try {
+            process0(evt);
+
+            onProcessEnd();
+        }
+        catch (Throwable e) {
+            onProcessError(e);
+        }
+    }
+
+    /**
+     * @param evt Event.
+     * @throws Exception If failed.
+     */
+    protected abstract void process0(WatchedEvent evt) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAliveNodeData.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAliveNodeData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAliveNodeData.java
new file mode 100644
index 0000000..d824377
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAliveNodeData.java
@@ -0,0 +1,40 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Zk Alive Node Data.
+ */
+public class ZkAliveNodeData implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    long lastProcEvt = -1;
+
+    /** */
+    transient boolean needUpdate;
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(ZkAliveNodeData.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkBulkJoinContext.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkBulkJoinContext.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkBulkJoinContext.java
new file mode 100644
index 0000000..a186aed
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkBulkJoinContext.java
@@ -0,0 +1,50 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.internal.util.typedef.T2;
+
+/**
+ *
+ */
+class ZkBulkJoinContext {
+    /** */
+    List<T2<ZkJoinedNodeEvtData, Map<Integer, Serializable>>> nodes;
+
+    /**
+     * @param nodeEvtData Node event data.
+     * @param discoData Discovery data for node.
+     */
+    void addJoinedNode(ZkJoinedNodeEvtData nodeEvtData, Map<Integer, Serializable> discoData) {
+        if (nodes == null)
+            nodes = new ArrayList<>();
+
+        nodes.add(new T2<>(nodeEvtData, discoData));
+    }
+
+    /**
+     * @return Number of joined nodes.
+     */
+    int nodes() {
+        return nodes != null ? nodes.size() : 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkClusterNodes.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkClusterNodes.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkClusterNodes.java
new file mode 100644
index 0000000..7e2ea7b
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkClusterNodes.java
@@ -0,0 +1,103 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.cluster.ClusterNode;
+
+/**
+ * Zk Cluster Nodes.
+ */
+public class ZkClusterNodes {
+    /** */
+    final ConcurrentSkipListMap<Long, ZookeeperClusterNode> nodesByOrder = new ConcurrentSkipListMap<>();
+
+    /** */
+    final ConcurrentSkipListMap<Long, ZookeeperClusterNode> nodesByInternalId = new ConcurrentSkipListMap<>();
+
+    /** */
+    final ConcurrentHashMap<UUID, ZookeeperClusterNode> nodesById = new ConcurrentHashMap<>();
+
+    /**
+     * @return Remote nodes.
+     */
+    public Collection<ClusterNode> remoteNodes() {
+        List<ClusterNode> nodes = new ArrayList<>();
+
+        for (ClusterNode node : nodesById.values()) {
+            if (!node.isLocal())
+                nodes.add(node);
+        }
+
+        return nodes;
+    }
+
+    /**
+     * @return Current nodes in topology.
+     */
+    @SuppressWarnings("unchecked")
+    List<ClusterNode> topologySnapshot() {
+        return new ArrayList<>((Collection)nodesByOrder.values());
+    }
+
+    /**
+     * @param node New node.
+     */
+    void addNode(ZookeeperClusterNode node) {
+        assert node.id() != null : node;
+        assert node.order() > 0 : node;
+
+        ZookeeperClusterNode old = nodesById.put(node.id(), node);
+
+        assert old == null : old;
+
+        old = nodesByOrder.put(node.order(), node);
+
+        assert old == null : old;
+
+        old = nodesByInternalId.put(node.internalId(), node);
+
+        assert old == null : old;
+    }
+
+    /**
+     * @param internalId Node internal ID.
+     * @return Removed node.
+     */
+    ZookeeperClusterNode removeNode(long internalId) {
+        ZookeeperClusterNode node = nodesByInternalId.remove(internalId);
+
+        assert node != null : internalId;
+        assert node.order() > 0 : node;
+
+        Object rvmd = nodesByOrder.remove(node.order());
+
+        assert rvmd != null;
+
+        rvmd = nodesById.remove(node.id());
+
+        assert rvmd != null;
+
+        return node;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorNodeState.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorNodeState.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorNodeState.java
new file mode 100644
index 0000000..9c21f13
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorNodeState.java
@@ -0,0 +1,46 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import java.util.BitSet;
+
+/**
+ *
+ */
+class ZkCommunicationErrorNodeState implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    final BitSet commState;
+
+    /** */
+    final Exception err;
+
+    /**
+     * @param commState Communication state.
+     * @param err Error if failed get communication state..
+     */
+    ZkCommunicationErrorNodeState(BitSet commState, Exception err) {
+        assert commState != null || err != null;
+
+        this.commState = commState;
+        this.err = err;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorProcessFuture.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorProcessFuture.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorProcessFuture.java
new file mode 100644
index 0000000..accda6e
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorProcessFuture.java
@@ -0,0 +1,411 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.spi.IgniteSpiTimeoutObject;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.jboss.netty.util.internal.ConcurrentHashMap;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Future is created on each node when either connection error occurs or resolve communication error request
+ * received.
+ */
+class ZkCommunicationErrorProcessFuture extends GridFutureAdapter<Void> implements IgniteSpiTimeoutObject, Runnable {
+    /** */
+    private final ZookeeperDiscoveryImpl impl;
+
+    /** */
+    private final IgniteLogger log;
+
+    /** */
+    private final Map<Long, GridFutureAdapter<Boolean>> nodeFuts = new ConcurrentHashMap<>();
+
+    /** */
+    private final long endTime;
+
+    /** */
+    private final IgniteUuid id;
+
+    /** */
+    private State state;
+
+    /** */
+    private long resolveTopVer;
+
+    /** */
+    private Set<Long> resFailedNodes;
+
+    /** */
+    private Exception resErr;
+
+    /** */
+    private ZkDistributedCollectDataFuture collectResFut;
+
+    /**
+     * @param impl Discovery impl.
+     * @param timeout Timeout to wait before initiating resolve process.
+     * @return Future.
+     */
+    static ZkCommunicationErrorProcessFuture createOnCommunicationError(ZookeeperDiscoveryImpl impl, long timeout) {
+        return new ZkCommunicationErrorProcessFuture(impl, State.WAIT_TIMEOUT, timeout);
+    }
+
+    /**
+     * @param impl Discovery impl.
+     * @return Future.
+     */
+    static ZkCommunicationErrorProcessFuture createOnStartResolveRequest(ZookeeperDiscoveryImpl impl) {
+        return new ZkCommunicationErrorProcessFuture(impl, State.RESOLVE_STARTED, 0);
+    }
+
+    /**
+     * @param impl Discovery implementation.
+     * @param state Initial state.
+     * @param timeout Wait timeout before initiating communication errors resolve.
+     */
+    private ZkCommunicationErrorProcessFuture(ZookeeperDiscoveryImpl impl, State state, long timeout) {
+        assert state != State.DONE;
+
+        this.impl = impl;
+        this.log = impl.log();
+
+        if (state == State.WAIT_TIMEOUT) {
+            assert timeout > 0 : timeout;
+
+            id = IgniteUuid.fromUuid(impl.localNode().id());
+            endTime = System.currentTimeMillis() + timeout;
+        }
+        else {
+            id = null;
+            endTime = 0;
+        }
+
+        this.state = state;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public IgniteLogger logger() {
+        return log;
+    }
+
+    /**
+     * @param collectResFut Collect nodes' communication status future.
+     */
+    void nodeResultCollectFuture(ZkDistributedCollectDataFuture collectResFut) {
+        assert this.collectResFut == null : collectResFut;
+
+        this.collectResFut = collectResFut;
+    }
+
+    /**
+     * @param top Topology.
+     * @throws Exception If failed.
+     */
+    void onTopologyChange(ZkClusterNodes top) throws Exception {
+        for (Map.Entry<Long, GridFutureAdapter<Boolean>> e : nodeFuts.entrySet()) {
+            if (!top.nodesByOrder.containsKey(e.getKey()))
+                e.getValue().onDone(false);
+        }
+
+        if (collectResFut != null)
+            collectResFut.onTopologyChange(top);
+    }
+
+    /**
+     * @param rtState Runtime state.
+     * @param futPath Future path.
+     * @param nodes Nodes to ping.
+     */
+    void checkConnection(final ZkRuntimeState rtState, final String futPath, List<ClusterNode> nodes) {
+        final TcpCommunicationSpi spi = (TcpCommunicationSpi)impl.spi.ignite().configuration().getCommunicationSpi();
+
+        IgniteFuture<BitSet> fut = spi.checkConnection(nodes);
+
+        fut.listen(new IgniteInClosure<IgniteFuture<BitSet>>() {
+            @Override public void apply(final IgniteFuture<BitSet> fut) {
+                // Future completed either from NIO thread or timeout worker, save result from another thread.
+                impl.runInWorkerThread(new ZkRunnable(rtState, impl) {
+                    @Override public void run0() throws Exception {
+                        BitSet commState = null;
+                        Exception err = null;
+
+                        try {
+                            commState = fut.get();
+                        }
+                        catch (Exception e) {
+                            err = e;
+                        }
+
+                        ZkCommunicationErrorNodeState state = new ZkCommunicationErrorNodeState(commState, err);
+
+                        ZkDistributedCollectDataFuture.saveNodeResult(futPath,
+                            rtState.zkClient,
+                            impl.localNode().order(),
+                            impl.marshalZip(state));
+                    }
+
+                    @Override void onStartFailed() {
+                        onError(rtState.errForClose);
+                    }
+                });
+
+            }
+        });
+    }
+
+    /**
+     *
+     */
+    void scheduleCheckOnTimeout() {
+        synchronized (this) {
+            if (state == State.WAIT_TIMEOUT)
+                impl.spi.getSpiContext().addTimeoutObject(this);
+        }
+    }
+
+    /**
+     * @param topVer Topology version.
+     * @return {@code False} if future was already completed and need create another future instance.
+     */
+    boolean onStartResolveRequest(long topVer) {
+        synchronized (this) {
+            if (state == State.DONE)
+                return false;
+
+            if (state == State.WAIT_TIMEOUT)
+                impl.spi.getSpiContext().removeTimeoutObject(this);
+
+            assert resolveTopVer == 0 : resolveTopVer;
+
+            resolveTopVer = topVer;
+
+            state = State.RESOLVE_STARTED;
+        }
+
+        return true;
+    }
+
+    /**
+     * @param err Error.
+     */
+    void onError(Exception err) {
+        assert err != null;
+
+        Map<Long, GridFutureAdapter<Boolean>> futs;
+
+        synchronized (this) {
+            if (state == State.DONE) {
+                assert resErr != null;
+
+                return;
+            }
+
+            state = State.DONE;
+
+            resErr = err;
+
+            futs = nodeFuts; // nodeFuts should not be modified after state changed to DONE.
+        }
+
+        for (Map.Entry<Long, GridFutureAdapter<Boolean>> e : futs.entrySet())
+            e.getValue().onDone(err);
+
+        onDone(err);
+    }
+
+    /**
+     * @param failedNodes Node failed as result of resolve process.
+     */
+    void onFinishResolve(Set<Long> failedNodes) {
+        Map<Long, GridFutureAdapter<Boolean>> futs;
+
+        synchronized (this) {
+            if (state == State.DONE) {
+                assert resErr != null;
+
+                return;
+            }
+
+            assert state == State.RESOLVE_STARTED : state;
+
+            state = State.DONE;
+
+            resFailedNodes = failedNodes;
+
+            futs = nodeFuts; // nodeFuts should not be modified after state changed to DONE.
+        }
+
+        for (Map.Entry<Long, GridFutureAdapter<Boolean>> e : futs.entrySet()) {
+            Boolean res = !F.contains(resFailedNodes, e.getKey());
+
+            e.getValue().onDone(res);
+        }
+
+        onDone();
+    }
+
+    /**
+     * @param node Node.
+     * @return Future finished when communication error resolve is done or {@code null} if another
+     *      resolve process should be started.
+     */
+    @Nullable IgniteInternalFuture<Boolean> nodeStatusFuture(ClusterNode node) {
+        GridFutureAdapter<Boolean> fut;
+
+        synchronized (this) {
+            if (state == State.DONE) {
+                if (resolveTopVer != 0 && node.order() <= resolveTopVer) {
+                    Boolean res = !F.contains(resFailedNodes, node.order());
+
+                    return new GridFinishedFuture<>(res);
+                }
+                else
+                    return null;
+            }
+
+            fut = nodeFuts.get(node.order());
+
+            if (fut == null)
+                nodeFuts.put(node.order(), fut = new GridFutureAdapter<>());
+        }
+
+        if (impl.node(node.order()) == null)
+            fut.onDone(false);
+
+        return fut;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void run() {
+        // Run from zk discovery worker pool after timeout.
+        if (needProcessTimeout()) {
+            try {
+                UUID reqId = UUID.randomUUID();
+
+                if (log.isInfoEnabled()) {
+                    log.info("Initiate cluster-wide communication error resolve process [reqId=" + reqId +
+                        ", errNodes=" + nodeFuts.size() + ']');
+                }
+
+                impl.sendCustomMessage(new ZkCommunicationErrorResolveStartMessage(reqId));
+            }
+            catch (Exception e) {
+                Collection<GridFutureAdapter<Boolean>> futs;
+
+                synchronized (this) {
+                    if (state != State.WAIT_TIMEOUT)
+                        return;
+
+                    state = State.DONE;
+                    resErr = e;
+
+                    futs = nodeFuts.values(); // nodeFuts should not be modified after state changed to DONE.
+                }
+
+                for (GridFutureAdapter<Boolean> fut : futs)
+                    fut.onDone(e);
+
+                onDone(e);
+            }
+        }
+    }
+
+    /**
+     * @return {@code True} if need initiate resolve process after timeout expired.
+     */
+    private boolean needProcessTimeout() {
+        synchronized (this) {
+            if (state != State.WAIT_TIMEOUT)
+                return false;
+
+            for (GridFutureAdapter<Boolean> fut : nodeFuts.values()) {
+                if (!fut.isDone())
+                    return true;
+            }
+
+            state = State.DONE;
+        }
+
+        onDone(null, null);
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteUuid id() {
+        return id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long endTime() {
+        return endTime;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onTimeout() {
+        if (needProcessTimeout())
+            impl.runInWorkerThread(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onDone(@Nullable Void res, @Nullable Throwable err) {
+        if (super.onDone(res, err)) {
+            impl.clearCommunicationErrorProcessFuture(this);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(ZkCommunicationErrorProcessFuture.class, this);
+    }
+
+    /**
+     *
+     */
+    enum State {
+        /** */
+        DONE,
+
+        /** */
+        WAIT_TIMEOUT,
+
+        /** */
+        RESOLVE_STARTED
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveFinishMessage.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveFinishMessage.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveFinishMessage.java
new file mode 100644
index 0000000..9b7476c
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveFinishMessage.java
@@ -0,0 +1,69 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.UUID;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+class ZkCommunicationErrorResolveFinishMessage implements DiscoverySpiCustomMessage, ZkInternalMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    final UUID futId;
+
+    /** */
+    final long topVer;
+
+    /** */
+    transient ZkCommunicationErrorResolveResult res;
+
+    /**
+     * @param futId Future ID.
+     * @param topVer Topology version when resolve process finished.
+     */
+    ZkCommunicationErrorResolveFinishMessage(UUID futId, long topVer) {
+        this.futId = futId;
+        this.topVer = topVer;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public DiscoverySpiCustomMessage ackMessage() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isMutable() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(ZkCommunicationErrorResolveFinishMessage.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveResult.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveResult.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveResult.java
new file mode 100644
index 0000000..23495aa
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveResult.java
@@ -0,0 +1,45 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.util.GridLongList;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+class ZkCommunicationErrorResolveResult implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    final GridLongList killedNodes;
+
+    /** */
+    final Exception err;
+
+    /**
+     * @param killedNodes Killed nodes.
+     * @param err Error.
+     */
+    ZkCommunicationErrorResolveResult(@Nullable GridLongList killedNodes, Exception err) {
+        this.killedNodes = killedNodes;
+        this.err = err;
+    }
+}


[10/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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 01a5a71..cebe4b1 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
@@ -56,6 +56,8 @@ import org.apache.ignite.internal.managers.deployment.GridDeployment;
 import org.apache.ignite.internal.managers.deployment.GridDeploymentInfo;
 import org.apache.ignite.internal.managers.deployment.GridDeploymentInfoBean;
 import org.apache.ignite.internal.managers.discovery.CustomEventListener;
+import org.apache.ignite.internal.managers.discovery.DiscoCache;
+import org.apache.ignite.internal.managers.discovery.DiscoveryMessageResultsCollector;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.managers.eventstorage.HighPriorityListener;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
@@ -63,6 +65,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionPartialCountersMap;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryHandler;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
@@ -147,6 +150,12 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
     /** Query sequence number for message topic. */
     private final AtomicLong seq = new AtomicLong();
 
+    /** */
+    private ContinuousRoutinesInfo routinesInfo;
+
+    /** */
+    private int discoProtoVer;
+
     /**
      * @param ctx Kernal context.
      */
@@ -156,6 +165,11 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
     /** {@inheritDoc} */
     @Override public void start() throws IgniteCheckedException {
+        discoProtoVer = ctx.discovery().mutableCustomMessages() ? 1 : 2;
+
+        if (discoProtoVer == 2)
+            routinesInfo = new ContinuousRoutinesInfo();
+
         if (ctx.config().isDaemon())
             return;
 
@@ -177,6 +191,8 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                 @Override public void onCustomEvent(AffinityTopologyVersion topVer,
                     ClusterNode snd,
                     StartRoutineDiscoveryMessage msg) {
+                    assert discoProtoVer == 1 : discoProtoVer;
+
                     if (ctx.isStopping())
                         return;
 
@@ -184,6 +200,20 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                 }
             });
 
+        ctx.discovery().setCustomEventListener(StartRoutineDiscoveryMessageV2.class,
+            new CustomEventListener<StartRoutineDiscoveryMessageV2>() {
+                @Override public void onCustomEvent(AffinityTopologyVersion topVer,
+                    ClusterNode snd,
+                    StartRoutineDiscoveryMessageV2 msg) {
+                    assert discoProtoVer == 2 : discoProtoVer;
+
+                    if (ctx.isStopping())
+                        return;
+
+                    processStartRequestV2(topVer, snd, msg);
+                }
+            });
+
         ctx.discovery().setCustomEventListener(StartRoutineAckDiscoveryMessage.class,
             new CustomEventListener<StartRoutineAckDiscoveryMessage>() {
                 @Override public void onCustomEvent(AffinityTopologyVersion topVer,
@@ -201,6 +231,9 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                 @Override public void onCustomEvent(AffinityTopologyVersion topVer,
                     ClusterNode snd,
                     StopRoutineDiscoveryMessage msg) {
+                    if (discoProtoVer == 2)
+                        routinesInfo.removeRoutine(msg.routineId);
+
                     if (ctx.isStopping())
                         return;
 
@@ -222,32 +255,36 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
         ctx.io().addMessageListener(TOPIC_CONTINUOUS, new GridMessageListener() {
             @Override public void onMessage(UUID nodeId, Object obj, byte plc) {
-                GridContinuousMessage msg = (GridContinuousMessage)obj;
+                if (obj instanceof ContinuousRoutineStartResultMessage)
+                    processRoutineStartResultMessage(nodeId, (ContinuousRoutineStartResultMessage)obj);
+                else {
+                    GridContinuousMessage msg = (GridContinuousMessage)obj;
 
-                if (msg.data() == null && msg.dataBytes() != null) {
-                    try {
-                        msg.data(U.unmarshal(marsh, msg.dataBytes(), U.resolveClassLoader(ctx.config())));
-                    }
-                    catch (IgniteCheckedException e) {
-                        U.error(log, "Failed to process message (ignoring): " + msg, e);
+                    if (msg.data() == null && msg.dataBytes() != null) {
+                        try {
+                            msg.data(U.unmarshal(marsh, msg.dataBytes(), U.resolveClassLoader(ctx.config())));
+                        }
+                        catch (IgniteCheckedException e) {
+                            U.error(log, "Failed to process message (ignoring): " + msg, e);
 
-                        return;
+                            return;
+                        }
                     }
-                }
 
-                switch (msg.type()) {
-                    case MSG_EVT_NOTIFICATION:
-                        processNotification(nodeId, msg);
+                    switch (msg.type()) {
+                        case MSG_EVT_NOTIFICATION:
+                            processNotification(nodeId, msg);
 
-                        break;
+                            break;
 
-                    case MSG_EVT_ACK:
-                        processMessageAck(msg);
+                        case MSG_EVT_ACK:
+                            processMessageAck(msg);
 
-                        break;
+                            break;
 
-                    default:
-                        assert false : "Unexpected message received: " + msg.type();
+                        default:
+                            assert false : "Unexpected message received: " + msg.type();
+                    }
                 }
             }
         });
@@ -341,6 +378,15 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
     /** {@inheritDoc} */
     @Override public void collectJoiningNodeData(DiscoveryDataBag dataBag) {
+        if (ctx.isDaemon())
+            return;
+
+        if (discoProtoVer == 2) {
+            routinesInfo.collectJoiningNodeData(dataBag);
+
+            return;
+        }
+
         Serializable data = getDiscoveryData(dataBag.joiningNodeId());
 
         if (data != null)
@@ -349,6 +395,15 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
     /** {@inheritDoc} */
     @Override public void collectGridNodeData(DiscoveryDataBag dataBag) {
+        if (ctx.isDaemon())
+            return;
+
+        if (discoProtoVer == 2) {
+            routinesInfo.collectGridNodeData(dataBag);
+
+            return;
+        }
+
         Serializable data = getDiscoveryData(dataBag.joiningNodeId());
 
         if (data != null)
@@ -393,6 +448,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
             return data;
         }
+
         return null;
     }
 
@@ -430,22 +486,118 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
     @Override public void onJoiningNodeDataReceived(JoiningNodeDiscoveryData data) {
         if (log.isDebugEnabled()) {
             log.debug("onJoiningNodeDataReceived [joining=" + data.joiningNodeId() +
-                    ", loc=" + ctx.localNodeId() +
-                    ", data=" + data.joiningNodeData() +
-                    ']');
+                ", loc=" + ctx.localNodeId() +
+                ", data=" + data.joiningNodeData() +
+                ']');
         }
 
-        if (data.hasJoiningNodeData())
-            onDiscoDataReceived((DiscoveryData) data.joiningNodeData());
+        if (discoProtoVer == 2) {
+            if (data.hasJoiningNodeData())    {
+                ContinuousRoutinesJoiningNodeDiscoveryData nodeData = (ContinuousRoutinesJoiningNodeDiscoveryData)
+                    data.joiningNodeData();
+
+                for (ContinuousRoutineInfo routineInfo : nodeData.startedRoutines) {
+                    routinesInfo.addRoutineInfo(routineInfo);
+
+                    startDiscoveryDataRoutine(routineInfo);
+                }
+            }
+        }
+        else {
+            if (data.hasJoiningNodeData())
+                onDiscoDataReceived((DiscoveryData) data.joiningNodeData());
+        }
     }
 
     /** {@inheritDoc} */
     @Override public void onGridDataReceived(GridDiscoveryData data) {
-        Map<UUID, Serializable> nodeSpecData = data.nodeSpecificData();
+        if (discoProtoVer == 2) {
+            if (ctx.isDaemon())
+                return;
+
+            if (data.commonData() != null) {
+                ContinuousRoutinesCommonDiscoveryData commonData =
+                    (ContinuousRoutinesCommonDiscoveryData)data.commonData();
+
+                for (ContinuousRoutineInfo routineInfo : commonData.startedRoutines) {
+                    if (routinesInfo.routineExists(routineInfo.routineId))
+                        continue;
+
+                    routinesInfo.addRoutineInfo(routineInfo);
+
+                    startDiscoveryDataRoutine(routineInfo);
+                }
+            }
+        }
+        else {
+            Map<UUID, Serializable> nodeSpecData = data.nodeSpecificData();
+
+            if (nodeSpecData != null) {
+                for (Map.Entry<UUID, Serializable> e : nodeSpecData.entrySet())
+                    onDiscoDataReceived((DiscoveryData) e.getValue());
+            }
+        }
+    }
+
+    /**
+     * @param routineInfo Routine info.
+     */
+    private void startDiscoveryDataRoutine(ContinuousRoutineInfo routineInfo) {
+        IgnitePredicate<ClusterNode> nodeFilter = null;
+
+        try {
+            if (routineInfo.nodeFilter != null) {
+                nodeFilter = U.unmarshal(marsh, routineInfo.nodeFilter, U.resolveClassLoader(ctx.config()));
+
+                ctx.resource().injectGeneric(nodeFilter);
+            }
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to unmarshal continuous routine filter, ignore routine [" +
+                "routineId=" + routineInfo.routineId +
+                ", srcNodeId=" + routineInfo.srcNodeId + ']', e);
+
+            return;
+        }
+
+        if (nodeFilter == null || nodeFilter.apply(ctx.discovery().localNode())) {
+            GridContinuousHandler hnd;
 
-        if (nodeSpecData != null) {
-            for (Map.Entry<UUID, Serializable> e : nodeSpecData.entrySet())
-                onDiscoDataReceived((DiscoveryData) e.getValue());
+            try {
+                hnd = U.unmarshal(marsh, routineInfo.hnd, U.resolveClassLoader(ctx.config()));
+
+                if (ctx.config().isPeerClassLoadingEnabled())
+                    hnd.p2pUnmarshal(routineInfo.srcNodeId, ctx);
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to unmarshal continuous routine handler, ignore routine [" +
+                    "routineId=" + routineInfo.routineId +
+                    ", srcNodeId=" + routineInfo.srcNodeId + ']', e);
+
+                return;
+            }
+
+            try {
+                registerHandler(routineInfo.srcNodeId,
+                    routineInfo.routineId,
+                    hnd,
+                    routineInfo.bufSize,
+                    routineInfo.interval,
+                    routineInfo.autoUnsubscribe,
+                    false);
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to register continuous routine handler, ignore routine [" +
+                    "routineId=" + routineInfo.routineId +
+                    ", srcNodeId=" + routineInfo.srcNodeId + ']', e);
+            }
+        }
+        else {
+            if (log.isDebugEnabled()) {
+                log.debug("Do not register continuous routine, rejected by node filter [" +
+                    "routineId=" + routineInfo.routineId +
+                    ", srcNodeId=" + routineInfo.srcNodeId + ']');
+            }
         }
     }
 
@@ -564,13 +716,14 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
      * @param rmtFilter Remote filter.
      * @param prjPred Projection predicate.
      * @return Routine ID.
+     * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
     public UUID registerStaticRoutine(
         String cacheName,
         CacheEntryUpdatedListener<?, ?> locLsnr,
         CacheEntryEventSerializableFilter rmtFilter,
-        @Nullable IgnitePredicate<ClusterNode> prjPred) {
+        @Nullable IgnitePredicate<ClusterNode> prjPred) throws IgniteCheckedException {
         String topicPrefix = "CONTINUOUS_QUERY_STATIC" + "_" + cacheName;
 
         CacheContinuousQueryHandler hnd = new CacheContinuousQueryHandler(
@@ -589,6 +742,17 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
         LocalRoutineInfo routineInfo = new LocalRoutineInfo(prjPred, hnd, 1, 0, true);
 
+        if (discoProtoVer == 2) {
+            routinesInfo.addRoutineInfo(createRoutineInfo(
+                ctx.localNodeId(),
+                routineId,
+                hnd,
+                prjPred,
+                routineInfo.bufSize,
+                routineInfo.interval,
+                routineInfo.autoUnsubscribe));
+        }
+
         locInfos.put(routineId, routineInfo);
 
         registerMessageListener(hnd);
@@ -597,6 +761,40 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param srcNodeId Source node ID.
+     * @param routineId Routine ID.
+     * @param hnd Handler.
+     * @param nodeFilter Node filter.
+     * @param bufSize Handler buffer size.
+     * @param interval Time interval.
+     * @param autoUnsubscribe Auto unsubscribe flag.
+     * @return Routine info instance.
+     * @throws IgniteCheckedException If failed.
+     */
+    private ContinuousRoutineInfo createRoutineInfo(
+        UUID srcNodeId,
+        UUID routineId,
+        GridContinuousHandler hnd,
+        @Nullable IgnitePredicate<ClusterNode> nodeFilter,
+        int bufSize,
+        long interval,
+        boolean autoUnsubscribe)
+        throws IgniteCheckedException {
+        byte[] hndBytes = marsh.marshal(hnd);
+
+        byte[] filterBytes = nodeFilter != null ? marsh.marshal(nodeFilter) : null;
+
+        return new ContinuousRoutineInfo(
+            srcNodeId,
+            routineId,
+            hndBytes,
+            filterBytes,
+            bufSize,
+            interval,
+            autoUnsubscribe);
+    }
+
+    /**
      * @param hnd Handler.
      * @param bufSize Buffer size.
      * @param interval Time interval.
@@ -638,30 +836,10 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         // Whether local node is included in routine.
         boolean locIncluded = prjPred == null || prjPred.apply(ctx.discovery().localNode());
 
-        StartRequestData reqData = new StartRequestData(prjPred, hnd.clone(), bufSize, interval, autoUnsubscribe);
+        AbstractContinuousMessage msg;
 
         try {
-            if (ctx.config().isPeerClassLoadingEnabled()) {
-                // Handle peer deployment for projection predicate.
-                if (prjPred != null && !U.isGrid(prjPred.getClass())) {
-                    Class cls = U.detectClass(prjPred);
-
-                    String clsName = cls.getName();
-
-                    GridDeployment dep = ctx.deploy().deploy(cls, U.detectClassLoader(cls));
-
-                    if (dep == null)
-                        throw new IgniteDeploymentCheckedException("Failed to deploy projection predicate: " + prjPred);
-
-                    reqData.className(clsName);
-                    reqData.deploymentInfo(new GridDeploymentInfoBean(dep));
-
-                    reqData.p2pMarshal(marsh);
-                }
-
-                // Handle peer deployment for other handler-specific objects.
-                reqData.handler().p2pMarshal(ctx);
-            }
+            msg = createStartMessage(routineId, hnd, bufSize, interval, autoUnsubscribe, prjPred);
         }
         catch (IgniteCheckedException e) {
             return new GridFinishedFuture<>(e);
@@ -674,20 +852,26 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
             return new GridFinishedFuture<>(new NodeStoppingException("Failed to start continuous query (node is stopping)"));
 
         try {
-            StartFuture fut = new StartFuture(ctx, routineId);
+            StartFuture fut = new StartFuture(routineId);
 
             startFuts.put(routineId, fut);
 
             try {
-                if (locIncluded || hnd.isQuery())
-                    registerHandler(ctx.localNodeId(), routineId, hnd, bufSize, interval, autoUnsubscribe, true);
+                if (locIncluded || hnd.isQuery()) {
+                    registerHandler(ctx.localNodeId(),
+                        routineId,
+                        hnd,
+                        bufSize,
+                        interval,
+                        autoUnsubscribe,
+                        true);
+                }
 
-                ctx.discovery().sendCustomEvent(new StartRoutineDiscoveryMessage(routineId, reqData,
-                    reqData.handler().keepBinary()));
-            }
-            catch (IgniteCheckedException e) {
-                startFuts.remove(routineId);
-                locInfos.remove(routineId);
+            ctx.discovery().sendCustomEvent(msg);
+        }
+        catch (IgniteCheckedException e) {
+            startFuts.remove(routineId);
+            locInfos.remove(routineId);
 
                 unregisterHandler(routineId, hnd, true);
 
@@ -707,6 +891,92 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param routineId Routine ID.
+     * @param hnd Handler.
+     * @param bufSize Buffer size.
+     * @param interval Interval.
+     * @param autoUnsubscribe Auto unsubscribe flag.
+     * @param nodeFilter Node filter.
+     * @return Routine start message.
+     * @throws IgniteCheckedException If failed.
+     */
+    private AbstractContinuousMessage createStartMessage(UUID routineId,
+        GridContinuousHandler hnd,
+        int bufSize,
+        long interval,
+        boolean autoUnsubscribe,
+        @Nullable IgnitePredicate<ClusterNode> nodeFilter)
+        throws IgniteCheckedException
+    {
+        hnd = hnd.clone();
+
+        String clsName = null;
+        GridDeploymentInfoBean dep = null;
+
+        if (ctx.config().isPeerClassLoadingEnabled()) {
+            // Handle peer deployment for projection predicate.
+            if (nodeFilter != null && !U.isGrid(nodeFilter.getClass())) {
+                Class cls = U.detectClass(nodeFilter);
+
+                clsName = cls.getName();
+
+                GridDeployment dep0 = ctx.deploy().deploy(cls, U.detectClassLoader(cls));
+
+                if (dep0 == null)
+                    throw new IgniteDeploymentCheckedException("Failed to deploy projection predicate: " + nodeFilter);
+
+                dep = new GridDeploymentInfoBean(dep0);
+            }
+
+            // Handle peer deployment for other handler-specific objects.
+            hnd.p2pMarshal(ctx);
+        }
+
+        if (discoProtoVer == 1) {
+            StartRequestData reqData = new StartRequestData(
+                nodeFilter,
+                hnd,
+                bufSize,
+                interval,
+                autoUnsubscribe);
+
+            if (clsName != null) {
+                reqData.className(clsName);
+                reqData.deploymentInfo(dep);
+
+                reqData.p2pMarshal(marsh);
+            }
+
+            return new StartRoutineDiscoveryMessage(
+                routineId,
+                reqData,
+                reqData.handler().keepBinary());
+        }
+        else {
+            assert discoProtoVer == 2 : discoProtoVer;
+
+            byte[] nodeFilterBytes = nodeFilter != null ? U.marshal(marsh, nodeFilter) : null;
+            byte[] hndBytes =  U.marshal(marsh, hnd);
+
+            StartRequestDataV2 reqData = new StartRequestDataV2(nodeFilterBytes,
+                hndBytes,
+                bufSize,
+                interval,
+                autoUnsubscribe);
+
+            if (clsName != null) {
+                reqData.className(clsName);
+                reqData.deploymentInfo(dep);
+            }
+
+            return new StartRoutineDiscoveryMessageV2(
+                routineId,
+                reqData,
+                hnd.keepBinary());
+        }
+    }
+
+    /**
      * @param hnd Handler.
      */
     private void registerMessageListener(GridContinuousHandler hnd) {
@@ -760,29 +1030,38 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                     doStop = true;
             }
 
-            if (doStop) {
-                // Unregister routine locally.
-                LocalRoutineInfo routine = locInfos.remove(routineId);
-
-                // Finish if routine is not found (wrong ID is provided).
-                if (routine == null) {
-                    stopFuts.remove(routineId);
+        if (doStop) {
+            boolean stop = false;
 
-                    fut.onDone();
+            // Unregister routine locally.
+            LocalRoutineInfo routine = locInfos.remove(routineId);
 
-                    return fut;
-                }
+            if (routine != null) {
+                stop = true;
 
                 // Unregister handler locally.
                 unregisterHandler(routineId, routine.hnd, true);
+            }
 
-                try {
-                    ctx.discovery().sendCustomEvent(new StopRoutineDiscoveryMessage(routineId));
-                }
-                catch (IgniteCheckedException e) {
-                    fut.onDone(e);
+            if (!stop && discoProtoVer == 2)
+                stop = routinesInfo.routineExists(routineId);
+
+            // Finish if routine is not found (wrong ID is provided).
+            if (!stop) {
+                stopFuts.remove(routineId);
+
+                    fut.onDone();
+
+                    return fut;
                 }
 
+            try {
+                ctx.discovery().sendCustomEvent(new StopRoutineDiscoveryMessage(routineId));
+            }
+            catch (IgniteCheckedException e) {
+                fut.onDone(e);
+            }
+
                 if (ctx.isStopping())
                     fut.onDone();
             }
@@ -924,6 +1203,9 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
         clientInfos.clear();
 
+        if (discoProtoVer == 2)
+            routinesInfo.onClientDisconnected(locInfos.keySet());
+
         if (log.isDebugEnabled()) {
             log.debug("after onDisconnected [rmtInfos=" + rmtInfos +
                 ", locInfos=" + locInfos +
@@ -996,35 +1278,11 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         StartFuture fut = startFuts.remove(msg.routineId());
 
         if (fut != null) {
-            if (msg.errs().isEmpty()) {
-                LocalRoutineInfo routine = locInfos.get(msg.routineId());
-
-                // Update partition counters.
-                if (routine != null && routine.handler().isQuery()) {
-                    Map<UUID, Map<Integer, T2<Long, Long>>> cntrsPerNode = msg.updateCountersPerNode();
-                    Map<Integer, T2<Long, Long>> cntrs = msg.updateCounters();
-
-                    GridCacheAdapter<Object, Object> interCache =
-                        ctx.cache().internalCache(routine.handler().cacheName());
-
-                    GridCacheContext cctx = interCache != null ? interCache.context() : null;
-
-                    if (cctx != null && cntrsPerNode != null && !cctx.isLocal() && cctx.affinityNode())
-                        cntrsPerNode.put(ctx.localNodeId(),
-                            toCountersMap(cctx.topology().localUpdateCounters(false)));
-
-                    routine.handler().updateCounters(topVer, cntrsPerNode, cntrs);
-                }
-
-                fut.onRemoteRegistered();
-            }
-            else {
-                IgniteCheckedException firstEx = F.first(msg.errs().values());
-
-                fut.onDone(firstEx);
-
-                stopRoutine(msg.routineId());
-            }
+            fut.onAllRemoteRegistered(
+                topVer,
+                msg.errs(),
+                msg.updateCountersPerNode(),
+                msg.updateCounters());
         }
     }
 
@@ -1138,6 +1396,199 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param sndId Sender node ID.
+     * @param msg Message.
+     */
+    private void processRoutineStartResultMessage(UUID sndId, ContinuousRoutineStartResultMessage msg) {
+        StartFuture fut = startFuts.get(msg.routineId());
+
+        if (fut != null)
+            fut.onResult(sndId, msg);
+    }
+
+    /**
+     * @param topVer Current topology version.
+     * @param snd Sender.
+     * @param msg Start request.
+     */
+    private void processStartRequestV2(final AffinityTopologyVersion topVer,
+        final ClusterNode snd,
+        final StartRoutineDiscoveryMessageV2 msg) {
+        StartRequestDataV2 reqData = msg.startRequestData();
+
+        ContinuousRoutineInfo routineInfo = new ContinuousRoutineInfo(snd.id(),
+            msg.routineId(),
+            reqData.handlerBytes(),
+            reqData.nodeFilterBytes(),
+            reqData.bufferSize(),
+            reqData.interval(),
+            reqData.autoUnsubscribe());
+
+        routinesInfo.addRoutineInfo(routineInfo);
+
+        final DiscoCache discoCache = ctx.discovery().discoCache(topVer);
+
+        // Should not use marshaller and send messages from discovery thread.
+        ctx.getSystemExecutorService().execute(new Runnable() {
+            @Override public void run() {
+                if (snd.id().equals(ctx.localNodeId())) {
+                    StartFuture fut = startFuts.get(msg.routineId());
+
+                    if (fut != null)
+                        fut.initRemoteNodes(discoCache);
+
+                    return;
+                }
+
+                StartRequestDataV2 reqData = msg.startRequestData();
+
+                Exception err = null;
+
+                IgnitePredicate<ClusterNode> nodeFilter = null;
+
+                byte[] cntrs = null;
+
+                if (reqData.nodeFilterBytes() != null) {
+                    try {
+                        if (ctx.config().isPeerClassLoadingEnabled() && reqData.className() != null) {
+                            String clsName = reqData.className();
+                            GridDeploymentInfo depInfo = reqData.deploymentInfo();
+
+                            GridDeployment dep = ctx.deploy().getGlobalDeployment(depInfo.deployMode(),
+                                clsName,
+                                clsName,
+                                depInfo.userVersion(),
+                                snd.id(),
+                                depInfo.classLoaderId(),
+                                depInfo.participants(),
+                                null);
+
+                            if (dep == null) {
+                                throw new IgniteDeploymentCheckedException("Failed to obtain deployment " +
+                                    "for class: " + clsName);
+                            }
+
+                            nodeFilter = U.unmarshal(marsh,
+                                reqData.nodeFilterBytes(),
+                                U.resolveClassLoader(dep.classLoader(), ctx.config()));
+                        }
+                        else {
+                            nodeFilter = U.unmarshal(marsh,
+                                reqData.nodeFilterBytes(),
+                                U.resolveClassLoader(ctx.config()));
+                        }
+
+                        if (nodeFilter != null)
+                            ctx.resource().injectGeneric(nodeFilter);
+                    }
+                    catch (Exception e) {
+                        err = e;
+
+                        U.error(log, "Failed to unmarshal continuous routine filter [" +
+                            "routineId=" + msg.routineId +
+                            ", srcNodeId=" + snd.id() + ']', e);
+                    }
+                }
+
+                boolean register = err == null &&
+                    (nodeFilter == null || nodeFilter.apply(ctx.discovery().localNode()));
+
+                if (register) {
+                    try {
+                        GridContinuousHandler hnd = U.unmarshal(marsh,
+                            reqData.handlerBytes(),
+                            U.resolveClassLoader(ctx.config()));
+
+                        if (ctx.config().isPeerClassLoadingEnabled())
+                            hnd.p2pUnmarshal(snd.id(), ctx);
+
+                        if (msg.keepBinary()) {
+                            assert hnd instanceof CacheContinuousQueryHandler : hnd;
+
+                            ((CacheContinuousQueryHandler)hnd).keepBinary(true);
+                        }
+
+                        GridContinuousHandler hnd0 = hnd instanceof GridMessageListenHandler ?
+                            new GridMessageListenHandler((GridMessageListenHandler)hnd) :
+                            hnd;
+
+                        registerHandler(snd.id(),
+                            msg.routineId,
+                            hnd0,
+                            reqData.bufferSize(),
+                            reqData.interval(),
+                            reqData.autoUnsubscribe(),
+                            false);
+
+                        if (hnd0.isQuery()) {
+                            GridCacheProcessor proc = ctx.cache();
+
+                            if (proc != null) {
+                                GridCacheAdapter cache = ctx.cache().internalCache(hnd0.cacheName());
+
+                                if (cache != null && !cache.isLocal() && cache.context().userCache()) {
+                                    CachePartitionPartialCountersMap cntrsMap =
+                                        cache.context().topology().localUpdateCounters(false);
+
+                                    cntrs = U.marshal(marsh, cntrsMap);
+                                }
+                            }
+                        }
+                    }
+                    catch (Exception e) {
+                        err = e;
+
+                        U.error(log, "Failed to register continuous routine handler [" +
+                            "routineId=" + msg.routineId +
+                            ", srcNodeId=" + snd.id() + ']', e);
+                    }
+                }
+
+                sendMessageStartResult(snd, msg.routineId(), cntrs, err);
+            }
+        });
+    }
+
+    /**
+     * @param node Target node.
+     * @param routineId Routine ID.
+     * @param cntrsMapBytes Marshalled {@link CachePartitionPartialCountersMap}.
+     * @param err Start error if any.
+     */
+    private void sendMessageStartResult(final ClusterNode node,
+        final UUID routineId,
+        byte[] cntrsMapBytes,
+        final @Nullable Exception err)
+    {
+        byte[] errBytes = null;
+
+        if (err != null) {
+            try {
+                errBytes = U.marshal(marsh, err);
+            }
+            catch (Exception e) {
+                U.error(log, "Failed to marshal routine start error: " + e, e);
+            }
+        }
+
+        ContinuousRoutineStartResultMessage msg = new ContinuousRoutineStartResultMessage(routineId,
+            cntrsMapBytes,
+            errBytes,
+            err != null);
+
+        try {
+            ctx.io().sendToGridTopic(node, TOPIC_CONTINUOUS, msg, SYSTEM_POOL, null);
+        }
+        catch (ClusterTopologyCheckedException e) {
+            if (log.isDebugEnabled())
+                log.debug("Failed to send routine start result, node failed: " + e);
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send routine start result: " + e, e);
+        }
+    }
+
+    /**
      * @param msg Message.
      */
     private void processMessageAck(GridContinuousMessage msg) {
@@ -1455,6 +1906,13 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
             UUID nodeId = ((DiscoveryEvent)evt).eventNode().id();
 
+            if (discoProtoVer == 2) {
+                routinesInfo.onNodeFail(nodeId);
+
+                for (StartFuture fut : startFuts.values())
+                    fut.onNodeFail(nodeId);
+            }
+
             clientInfos.remove(nodeId);
 
             // Unregister handlers created by left node.
@@ -1894,10 +2352,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
     /**
      * Future for start routine.
      */
-    private static class StartFuture extends GridFutureAdapter<UUID> {
-        /** */
-        private GridKernalContext ctx;
-
+    private class StartFuture extends GridFutureAdapter<UUID> {
         /** Consume ID. */
         private UUID routineId;
 
@@ -1907,56 +2362,170 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         /** All remote listeners are registered. */
         private volatile boolean rmt;
 
-        /** Timeout object. */
-        private volatile GridTimeoutObject timeoutObj;
+        /** */
+        private final DiscoveryMessageResultsCollector<ContinuousRoutineStartResultMessage, RoutineRegisterResults>
+            resCollect;
 
         /**
-         * @param ctx Kernal context.
          * @param routineId Consume ID.
          */
-        StartFuture(GridKernalContext ctx, UUID routineId) {
-            this.ctx = ctx;
-
+        StartFuture(UUID routineId) {
             this.routineId = routineId;
+
+            resCollect = new DiscoveryMessageResultsCollector<ContinuousRoutineStartResultMessage, RoutineRegisterResults>(ctx) {
+                @Override protected RoutineRegisterResults createResult(Map<UUID, NodeMessage<ContinuousRoutineStartResultMessage>> rcvd) {
+                    Map<UUID, Exception> errs = null;
+                    Map<UUID, Map<Integer, T2<Long, Long>>> cntrsPerNode = null;
+
+                    for (Map.Entry<UUID, NodeMessage<ContinuousRoutineStartResultMessage>> entry : rcvd.entrySet()) {
+                        ContinuousRoutineStartResultMessage msg = entry.getValue().message();
+
+                        if (msg == null)
+                            continue;
+
+                        if (msg.error()) {
+                            byte[] errBytes = msg.errorBytes();
+
+                            Exception err = null;
+
+                            if (errBytes != null) {
+                                try {
+                                    err = U.unmarshal(marsh, errBytes, U.resolveClassLoader(ctx.config()));
+                                }
+                                catch (Exception e) {
+                                    U.warn(log, "Failed to unmarhal continuous routine start error: " + e);
+                                }
+                            }
+
+                            if (err == null) {
+                                err = new IgniteCheckedException("Failed to start continuous " +
+                                    "routine on node: " + entry.getKey());
+                            }
+
+                            if (errs == null)
+                                errs = new HashMap<>();
+
+                            errs.put(entry.getKey(), err);
+                        }
+                        else {
+                            byte[] cntrsMapBytes = msg.countersMapBytes();
+
+                            if (cntrsMapBytes != null) {
+                                try {
+                                    CachePartitionPartialCountersMap cntrsMap = U.unmarshal(
+                                        marsh,
+                                        cntrsMapBytes,
+                                        U.resolveClassLoader(ctx.config()));
+
+                                    if (cntrsPerNode == null)
+                                        cntrsPerNode = new HashMap<>();
+
+                                    cntrsPerNode.put(entry.getKey(), CachePartitionPartialCountersMap.toCountersMap(cntrsMap));
+                                }
+                                catch (Exception e) {
+                                    U.warn(log, "Failed to unmarhal continuous query update counters: " + e);
+                                }
+                            }
+                        }
+                    }
+
+                    return new RoutineRegisterResults(discoCache.version(), errs, cntrsPerNode);
+                }
+
+                @Override protected void onResultsCollected(RoutineRegisterResults res0) {
+                    onAllRemoteRegistered(res0.topVer, res0.errs, res0.cntrsPerNode, null);
+                }
+
+                @Override protected boolean waitForNode(DiscoCache discoCache, ClusterNode node) {
+                    return !ctx.localNodeId().equals(node.id());
+                }
+            };
         }
 
         /**
-         * Called when local listener is registered.
+         * @param topVer Topology version.
+         * @param errs Errors.
+         * @param cntrsPerNode Update counters.
+         * @param cntrs Update counters.
          */
-        public void onLocalRegistered() {
-            loc = true;
+        private void onAllRemoteRegistered(
+            AffinityTopologyVersion topVer,
+            @Nullable Map<UUID, ? extends Exception> errs,
+            Map<UUID, Map<Integer, T2<Long, Long>>> cntrsPerNode,
+            Map<Integer, T2<Long, Long>> cntrs) {
+            try {
+                if (errs == null || errs.isEmpty()) {
+                    LocalRoutineInfo routine = locInfos.get(routineId);
 
-            if (rmt && !isDone())
-                onDone(routineId);
+                    // Update partition counters.
+                    if (routine != null && routine.handler().isQuery()) {
+                        GridCacheAdapter<Object, Object> interCache =
+                            ctx.cache().internalCache(routine.handler().cacheName());
+
+                        GridCacheContext cctx = interCache != null ? interCache.context() : null;
+
+                        if (cctx != null && cntrsPerNode != null && !cctx.isLocal() && cctx.affinityNode())
+                            cntrsPerNode.put(ctx.localNodeId(),
+                                toCountersMap(cctx.topology().localUpdateCounters(false)));
+
+                        routine.handler().updateCounters(topVer, cntrsPerNode, cntrs);
+                    }
+
+                    onRemoteRegistered();
+                }
+                else {
+                    Exception firstEx = F.first(errs.values());
+
+                    onDone(firstEx);
+
+                    stopRoutine(routineId);
+                }
+            }
+            finally {
+                startFuts.remove(routineId, this);
+            }
         }
 
         /**
-         * Called when all remote listeners are registered.
+         * @param discoCache Discovery state.
          */
-        public void onRemoteRegistered() {
-            rmt = true;
+        void initRemoteNodes(DiscoCache discoCache) {
+            resCollect.init(discoCache);
+        }
 
-            if (loc && !isDone())
-                onDone(routineId);
+        /**
+         * @param nodeId Node ID.
+         * @param msg Message.
+         */
+        void onResult(UUID nodeId, ContinuousRoutineStartResultMessage msg) {
+            resCollect.onMessage(nodeId, msg);
         }
 
         /**
-         * @param timeoutObj Timeout object.
+         * @param nodeId Failed node ID.
          */
-        public void addTimeoutObject(GridTimeoutObject timeoutObj) {
-            assert timeoutObj != null;
+        void onNodeFail(UUID nodeId) {
+            resCollect.onNodeFail(nodeId);
+        }
 
-            this.timeoutObj = timeoutObj;
+        /**
+         * Called when local listener is registered.
+         */
+        void onLocalRegistered() {
+            loc = true;
 
-            ctx.timeout().addTimeoutObject(timeoutObj);
+            if (rmt && !isDone())
+                onDone(routineId);
         }
 
-        /** {@inheritDoc} */
-        @Override public boolean onDone(@Nullable UUID res, @Nullable Throwable err) {
-            if (timeoutObj != null)
-                ctx.timeout().removeTimeoutObject(timeoutObj);
+        /**
+         * Called when all remote listeners are registered.
+         */
+        void onRemoteRegistered() {
+            rmt = true;
 
-            return super.onDone(res, err);
+            if (loc && !isDone())
+                onDone(routineId);
         }
 
         /** {@inheritDoc} */
@@ -1966,6 +2535,33 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
     }
 
     /**
+     *
+     */
+    private static class RoutineRegisterResults {
+        /** */
+        private final AffinityTopologyVersion topVer;
+
+        /** */
+        private final Map<UUID, ? extends Exception> errs;
+
+        /** */
+        private final Map<UUID, Map<Integer, T2<Long, Long>>> cntrsPerNode;
+
+        /**
+         * @param topVer Topology version.
+         * @param errs Errors.
+         * @param cntrsPerNode Update counters.
+         */
+        RoutineRegisterResults(AffinityTopologyVersion topVer,
+            Map<UUID, ? extends Exception> errs,
+            Map<UUID, Map<Integer, T2<Long, Long>>> cntrsPerNode) {
+            this.topVer = topVer;
+            this.errs = errs;
+            this.cntrsPerNode = cntrsPerNode;
+        }
+    }
+
+    /**
      * Future for stop routine.
      */
     private static class StopFuture extends GridFutureAdapter<Object> {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRequestDataV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRequestDataV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRequestDataV2.java
new file mode 100644
index 0000000..c001616
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRequestDataV2.java
@@ -0,0 +1,164 @@
+/*
+ * 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.continuous;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.managers.deployment.GridDeploymentInfo;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Start request data.
+ */
+class StartRequestDataV2 implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Serialized node filter. */
+    private byte[] nodeFilterBytes;
+
+    /** Deployment class name. */
+    private String clsName;
+
+    /** Deployment info. */
+    private GridDeploymentInfo depInfo;
+
+    /** Serialized handler. */
+    private byte[] hndBytes;
+
+    /** Buffer size. */
+    private int bufSize;
+
+    /** Time interval. */
+    private long interval;
+
+    /** Automatic unsubscribe flag. */
+    private boolean autoUnsubscribe;
+
+    /**
+     * @param nodeFilterBytes Serialized node filter.
+     * @param hndBytes Serialized handler.
+     * @param bufSize Buffer size.
+     * @param interval Time interval.
+     * @param autoUnsubscribe Automatic unsubscribe flag.
+     */
+    StartRequestDataV2(
+        byte[] nodeFilterBytes,
+        byte[] hndBytes,
+        int bufSize,
+        long interval,
+        boolean autoUnsubscribe) {
+        assert hndBytes != null;
+        assert bufSize > 0;
+        assert interval >= 0;
+
+        this.nodeFilterBytes = nodeFilterBytes;
+        this.hndBytes = hndBytes;
+        this.bufSize = bufSize;
+        this.interval = interval;
+        this.autoUnsubscribe = autoUnsubscribe;
+    }
+
+    /**
+     * @return Serialized node filter.
+     */
+    public byte[] nodeFilterBytes() {
+        return nodeFilterBytes;
+    }
+
+    /**
+     * @return Deployment class name.
+     */
+    public String className() {
+        return clsName;
+    }
+
+    /**
+     * @param clsName New deployment class name.
+     */
+    public void className(String clsName) {
+        this.clsName = clsName;
+    }
+
+    /**
+     * @return Deployment info.
+     */
+    public GridDeploymentInfo deploymentInfo() {
+        return depInfo;
+    }
+
+    /**
+     * @param depInfo New deployment info.
+     */
+    public void deploymentInfo(GridDeploymentInfo depInfo) {
+        this.depInfo = depInfo;
+    }
+
+    /**
+     * @return Handler.
+     */
+    public byte[] handlerBytes() {
+        return hndBytes;
+    }
+
+    /**
+     * @return Buffer size.
+     */
+    public int bufferSize() {
+        return bufSize;
+    }
+
+    /**
+     * @param bufSize New buffer size.
+     */
+    public void bufferSize(int bufSize) {
+        this.bufSize = bufSize;
+    }
+
+    /**
+     * @return Time interval.
+     */
+    public long interval() {
+        return interval;
+    }
+
+    /**
+     * @param interval New time interval.
+     */
+    public void interval(long interval) {
+        this.interval = interval;
+    }
+
+    /**
+     * @return Automatic unsubscribe flag.
+     */
+    public boolean autoUnsubscribe() {
+        return autoUnsubscribe;
+    }
+
+    /**
+     * @param autoUnsubscribe New automatic unsubscribe flag.
+     */
+    public void autoUnsubscribe(boolean autoUnsubscribe) {
+        this.autoUnsubscribe = autoUnsubscribe;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(StartRequestDataV2.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessageV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessageV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessageV2.java
new file mode 100644
index 0000000..275765d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessageV2.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.continuous;
+
+import java.util.UUID;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+public class StartRoutineDiscoveryMessageV2 extends AbstractContinuousMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private static final int KEEP_BINARY_FLAG = 0x01;
+
+    /** */
+    private final StartRequestDataV2 startReqData;
+
+    /** Flags. */
+    private int flags;
+
+    /**
+     * @param routineId Routine id.
+     * @param startReqData Start request data.
+     * @param keepBinary Keep binary flag.
+     */
+    StartRoutineDiscoveryMessageV2(UUID routineId, StartRequestDataV2 startReqData, boolean keepBinary) {
+        super(routineId);
+
+        this.startReqData = startReqData;
+
+        if (keepBinary)
+            flags |= KEEP_BINARY_FLAG;
+    }
+
+    /**
+     * @return Start request data.
+     */
+    public StartRequestDataV2 startRequestData() {
+        return startReqData;
+    }
+
+    /**
+     * @return {@code True} if keep binary flag was set on continuous handler.
+     */
+    public boolean keepBinary() {
+        return (flags & KEEP_BINARY_FLAG) != 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public DiscoveryCustomMessage ackMessage() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(StartRoutineDiscoveryMessageV2.class, this, "routineId", routineId());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineAckDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineAckDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineAckDiscoveryMessage.java
index 79d8b29..dfba0e7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineAckDiscoveryMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineAckDiscoveryMessage.java
@@ -42,6 +42,11 @@ public class StopRoutineAckDiscoveryMessage extends AbstractContinuousMessage {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(StopRoutineAckDiscoveryMessage.class, this, "routineId", routineId());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/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 4a893f4..8cad342 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
@@ -47,6 +47,7 @@ import javax.cache.expiry.ExpiryPolicy;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteClientDisconnectedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteDataStreamerTimeoutException;
 import org.apache.ignite.IgniteException;
@@ -100,6 +101,7 @@ import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.GPC;
@@ -1059,6 +1061,9 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             return;
 
         while (true) {
+            if (disconnectErr != null)
+                throw disconnectErr;
+
             Queue<IgniteInternalFuture<?>> q = null;
 
             for (Buffer buf : bufMappings.values()) {
@@ -1826,15 +1831,19 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                 catch (IgniteCheckedException e) {
                     GridFutureAdapter<Object> fut0 = ((GridFutureAdapter<Object>)fut);
 
-                    try {
-                        if (ctx.discovery().alive(node) && ctx.discovery().pingNode(node.id()))
-                            fut0.onDone(e);
-                        else
-                            fut0.onDone(new ClusterTopologyCheckedException("Failed to send request (node has left): "
-                                + node.id()));
-                    }
-                    catch (IgniteClientDisconnectedCheckedException e0) {
-                        fut0.onDone(e0);
+                    if (X.hasCause(e, IgniteClientDisconnectedCheckedException.class, IgniteClientDisconnectedException.class))
+                        fut0.onDone(e);
+                    else {
+                        try {
+                            if (ctx.discovery().alive(node) && ctx.discovery().pingNode(node.id()))
+                                fut0.onDone(e);
+                            else
+                                fut0.onDone(new ClusterTopologyCheckedException("Failed to send request (node has left): "
+                                    + node.id()));
+                        }
+                        catch (IgniteClientDisconnectedCheckedException e0) {
+                            fut0.onDone(e0);
+                        }
                     }
                 }
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingAcceptedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingAcceptedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingAcceptedMessage.java
index 7af0559..80e3f7d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingAcceptedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingAcceptedMessage.java
@@ -63,6 +63,11 @@ public class MappingAcceptedMessage implements DiscoveryCustomMessage {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
         AffinityTopologyVersion topVer, DiscoCache discoCache) {
         throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingProposedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingProposedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingProposedMessage.java
index b4e13fb..9358585 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingProposedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingProposedMessage.java
@@ -98,6 +98,11 @@ public class MappingProposedMessage implements DiscoveryCustomMessage {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
         AffinityTopologyVersion topVer, DiscoCache discoCache) {
         throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaFinishDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaFinishDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaFinishDiscoveryMessage.java
index 2245b24..f802e09 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaFinishDiscoveryMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaFinishDiscoveryMessage.java
@@ -59,6 +59,11 @@ public class SchemaFinishDiscoveryMessage extends SchemaAbstractDiscoveryMessage
     }
 
     /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean exchange() {
         return false;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaProposeDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaProposeDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaProposeDiscoveryMessage.java
index 0e1270b..62b6d6a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaProposeDiscoveryMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaProposeDiscoveryMessage.java
@@ -60,6 +60,11 @@ public class SchemaProposeDiscoveryMessage extends SchemaAbstractDiscoveryMessag
     }
 
     /** {@inheritDoc} */
+    @Override public boolean stopProcess() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean exchange() {
         return exchange;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index e0ec8d1..0fcde0e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -491,6 +491,17 @@ public class GridNioServer<T> {
 
     /**
      * @param ses Session.
+     */
+    public void closeFromWorkerThread(GridNioSession ses) {
+        assert ses instanceof GridSelectorNioSessionImpl : ses;
+
+        GridSelectorNioSessionImpl ses0 = (GridSelectorNioSessionImpl)ses;
+
+        ((AbstractNioClientWorker)ses0.worker()).close((GridSelectorNioSessionImpl)ses, null);
+    }
+
+    /**
+     * @param ses Session.
      * @param msg Message.
      * @param createFut {@code True} if future should be created.
      * @param ackC Closure invoked when message ACK is received.
@@ -2170,7 +2181,12 @@ public class GridNioServer<T> {
                 dumpSelectorInfo(sb, keys);
 
             for (SelectionKey key : keys) {
-                GridSelectorNioSessionImpl ses = (GridSelectorNioSessionImpl)key.attachment();
+                GridNioKeyAttachment attach = (GridNioKeyAttachment)key.attachment();
+
+                if (!attach.hasSession())
+                    continue;
+
+                GridSelectorNioSessionImpl ses = attach.session();
 
                 boolean sesInfo = p == null || p.apply(ses);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
index 1754cc8..e8c27d2 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
@@ -963,5 +963,15 @@ public abstract class IgniteSpiAdapter implements IgniteSpi {
         @Override public Map<String, Object> nodeAttributes() {
             return Collections.emptyMap();
         }
+
+        /** {@inheritDoc} */
+        @Override public boolean communicationFailureResolveSupported() {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void resolveCommunicationFailure(ClusterNode node, Exception err) {
+            throw new UnsupportedOperationException();
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
index 108c4d4..d4402f4 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
@@ -365,4 +365,15 @@ public interface IgniteSpiContext {
      * @return Current node attributes.
      */
     public Map<String, Object> nodeAttributes();
+
+    /**
+     * @return {@code True} if cluster supports communication error resolving.
+     */
+    public boolean communicationFailureResolveSupported();
+
+    /**
+     * @param node Problem node.
+     * @param err Error.
+     */
+    public void resolveCommunicationFailure(ClusterNode node, Exception err);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index e1addd8..4a0710e 100755
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -32,6 +32,7 @@ import java.nio.channels.SocketChannel;
 import java.nio.channels.spi.AbstractInterruptibleChannel;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.BitSet;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -66,12 +67,14 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.managers.eventstorage.HighPriorityListener;
 import org.apache.ignite.internal.util.GridConcurrentFactory;
 import org.apache.ignite.internal.util.GridSpinReadWriteLock;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
 import org.apache.ignite.internal.util.ipc.IpcEndpoint;
 import org.apache.ignite.internal.util.ipc.IpcToNioAdapter;
 import org.apache.ignite.internal.util.ipc.shmem.IpcOutOfSystemResourcesException;
@@ -134,6 +137,9 @@ import org.apache.ignite.spi.IgniteSpiThread;
 import org.apache.ignite.spi.IgniteSpiTimeoutObject;
 import org.apache.ignite.spi.communication.CommunicationListener;
 import org.apache.ignite.spi.communication.CommunicationSpi;
+import org.apache.ignite.spi.communication.tcp.internal.ConnectionKey;
+import org.apache.ignite.spi.communication.tcp.internal.TcpCommunicationConnectionCheckFuture;
+import org.apache.ignite.spi.communication.tcp.internal.TcpCommunicationNodeConnectionCheckFuture;
 import org.apache.ignite.spi.communication.tcp.messages.HandshakeMessage;
 import org.apache.ignite.spi.communication.tcp.messages.HandshakeMessage2;
 import org.apache.ignite.spi.communication.tcp.messages.NodeIdMessage;
@@ -146,6 +152,7 @@ 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.util.nio.GridNioSessionMetaKey.SSL_META;
+import static org.apache.ignite.spi.communication.tcp.internal.TcpCommunicationConnectionCheckFuture.SES_FUT_META;
 import static org.apache.ignite.spi.communication.tcp.messages.RecoveryLastReceivedMessage.ALREADY_CONNECTED;
 import static org.apache.ignite.spi.communication.tcp.messages.RecoveryLastReceivedMessage.NEED_WAIT;
 import static org.apache.ignite.spi.communication.tcp.messages.RecoveryLastReceivedMessage.NODE_STOPPING;
@@ -310,7 +317,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
     private static final IgniteProductVersion VERSION_SINCE_CLIENT_COULD_WAIT_TO_CONNECT = IgniteProductVersion.fromString("2.1.4");
 
     /** Connection index meta for session. */
-    private static final int CONN_IDX_META = GridNioSessionMetaKey.nextUniqueKey();
+    public static final int CONN_IDX_META = GridNioSessionMetaKey.nextUniqueKey();
 
     /** Message tracker meta for session. */
     private static final int TRACKER_META = GridNioSessionMetaKey.nextUniqueKey();
@@ -408,6 +415,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
                 ConnectionKey connId = ses.meta(CONN_IDX_META);
 
                 if (connId != null) {
+                    if (connId.dummy())
+                        return;
+
                     UUID id = connId.nodeId();
 
                     GridCommunicationClient[] nodeClients = clients.get(id);
@@ -481,20 +491,22 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
                 if (rmtNode == null) {
                     DiscoverySpi discoverySpi = ignite().configuration().getDiscoverySpi();
 
-                    assert discoverySpi instanceof TcpDiscoverySpi;
-
-                    TcpDiscoverySpi tcpDiscoverySpi = (TcpDiscoverySpi) discoverySpi;
+                    boolean unknownNode = true;
 
-                    ClusterNode node0 = tcpDiscoverySpi.getNode0(sndId);
+                    if (discoverySpi instanceof TcpDiscoverySpi) {
+                        TcpDiscoverySpi tcpDiscoverySpi = (TcpDiscoverySpi) discoverySpi;
 
-                    boolean unknownNode = true;
+                        ClusterNode node0 = tcpDiscoverySpi.getNode0(sndId);
 
-                    if (node0 != null) {
-                        assert node0.isClient() : node0;
+                        if (node0 != null) {
+                            assert node0.isClient() : node0;
 
-                        if (node0.version().compareTo(VERSION_SINCE_CLIENT_COULD_WAIT_TO_CONNECT) >= 0)
-                            unknownNode = false;
+                            if (node0.version().compareTo(VERSION_SINCE_CLIENT_COULD_WAIT_TO_CONNECT) >= 0)
+                                unknownNode = false;
+                        }
                     }
+                    else if (discoverySpi instanceof IgniteDiscoverySpi)
+                        unknownNode = !((IgniteDiscoverySpi) discoverySpi).knownNode(sndId);
 
                     if (unknownNode) {
                         U.warn(log, "Close incoming connection, unknown node [nodeId=" + sndId + ", ses=" + ses + ']');
@@ -709,9 +721,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
                     }
                 }
                 else {
-                    metricsLsnr.onMessageReceived(msg, connKey.nodeId());
-
                     if (msg instanceof RecoveryLastReceivedMessage) {
+                        metricsLsnr.onMessageReceived(msg, connKey.nodeId());
+
                         GridNioRecoveryDescriptor recovery = ses.outRecoveryDescriptor();
 
                         if (recovery != null) {
@@ -724,9 +736,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
                             }
 
                             recovery.ackReceived(msg0.received());
-
-                            return;
                         }
+
+                        return;
                     }
                     else {
                         GridNioRecoveryDescriptor recovery = ses.inRecoveryDescriptor();
@@ -746,8 +758,23 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
                                 recovery.lastAcknowledged(rcvCnt);
                             }
                         }
+                        else if (connKey.dummy()) {
+                            assert msg instanceof NodeIdMessage : msg;
+
+                            TcpCommunicationNodeConnectionCheckFuture fut = ses.meta(SES_FUT_META);
+
+                            assert fut != null : msg;
+
+                            fut.onConnected(U.bytesToUuid(((NodeIdMessage)msg).nodeIdBytes(), 0));
+
+                            nioSrvr.closeFromWorkerThread(ses);
+
+                            return;
+                        }
                     }
 
+                    metricsLsnr.onMessageReceived(msg, connKey.nodeId());
+
                     IgniteRunnable c;
 
                     if (msgQueueLimit > 0) {
@@ -2112,6 +2139,13 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
         }
     }
 
+    /**
+     * @return Bound TCP server port.
+     */
+    public int boundPort() {
+        return boundTcpPort;
+    }
+
     /** {@inheritDoc} */
     @Override public void spiStart(String igniteInstanceName) throws IgniteSpiException {
         assert locHost != null;
@@ -2570,6 +2604,27 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
     }
 
     /**
+     * @param nodes Nodes to check connection with.
+     * @return Result future (each bit in result BitSet contains connection status to corresponding node).
+     */
+    public IgniteFuture<BitSet> checkConnection(List<ClusterNode> nodes) {
+        TcpCommunicationConnectionCheckFuture fut = new TcpCommunicationConnectionCheckFuture(
+            this,
+            log.getLogger(TcpCommunicationConnectionCheckFuture.class),
+            nioSrvr,
+            nodes);
+
+        long timeout = failureDetectionTimeoutEnabled() ? failureDetectionTimeout() : connTimeout;
+
+        if (log.isInfoEnabled())
+            log.info("Start check connection process [nodeCnt=" + nodes.size() + ", timeout=" + timeout + ']');
+
+        fut.init(timeout);
+
+        return new IgniteFutureImpl<>(fut);
+    }
+
+    /**
      * Sends given message to destination node. Note that characteristics of the
      * exchange such as durability, guaranteed delivery or error notification is
      * dependant on SPI implementation.
@@ -3010,7 +3065,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
             ConnectionKey id = ses.meta(CONN_IDX_META);
 
             if (id != null) {
-                ClusterNode node = getSpiContext().node(id.nodeId);
+                ClusterNode node = getSpiContext().node(id.nodeId());
 
                 if (node != null && node.isClient()) {
                     String msg = "Client node outbound message queue size exceeded slowClientQueueLimit, " +
@@ -3031,9 +3086,20 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
     /**
      * @param node Node.
      * @return Node addresses.
+     * @throws IgniteCheckedException If failed.
+     */
+    private Collection<InetSocketAddress> nodeAddresses(ClusterNode node) throws IgniteCheckedException {
+        return nodeAddresses(node, filterReachableAddresses);
+    }
+
+    /**
+     * @param node Node.
+     * @param filterReachableAddresses Filter addresses flag.
+     * @return Node addresses.
      * @throws IgniteCheckedException If node does not have addresses.
      */
-    private LinkedHashSet<InetSocketAddress> nodeAddresses(ClusterNode node) throws IgniteCheckedException {
+    public Collection<InetSocketAddress> nodeAddresses(ClusterNode node, boolean filterReachableAddresses)
+        throws IgniteCheckedException {
         Collection<String> rmtAddrs0 = node.attribute(createSpiAttributeName(ATTR_ADDRS));
         Collection<String> rmtHostNames0 = node.attribute(createSpiAttributeName(ATTR_HOST_NAMES));
         Integer boundPort = node.attribute(createSpiAttributeName(ATTR_PORT));
@@ -3114,7 +3180,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
      * @throws IgniteCheckedException If failed.
      */
     protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) throws IgniteCheckedException {
-        LinkedHashSet<InetSocketAddress> addrs = nodeAddresses(node);
+        Collection<InetSocketAddress> addrs = nodeAddresses(node);
 
         GridCommunicationClient client = null;
         IgniteCheckedException errs = null;
@@ -3132,6 +3198,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
             int lastWaitingTimeout = 1;
 
             while (client == null) { // Reconnection on handshake timeout.
+                if (stopping)
+                    throw new IgniteSpiException("Node is stopping.");
+
                 if (addr.getAddress().isLoopbackAddress() && addr.getPort() == boundTcpPort) {
                     if (log.isDebugEnabled())
                         log.debug("Skipping local address [addr=" + addr +
@@ -3372,8 +3441,18 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
                     "operating system firewall is disabled on local and remote hosts) " +
                     "[addrs=" + addrs + ']');
 
-            if (enableForcibleNodeKill) {
-                if (getSpiContext().node(node.id()) != null
+            boolean commErrResolve = false;
+
+            IgniteSpiContext ctx = getSpiContext();
+
+            if (connectionError(errs) && ctx.communicationFailureResolveSupported()) {
+                commErrResolve = true;
+
+                ctx.resolveCommunicationFailure(node, errs);
+            }
+
+            if (!commErrResolve && enableForcibleNodeKill) {
+                if (ctx.node(node.id()) != null
                     && (CU.clientNode(node) ||  !CU.clientNode(getLocalNode())) &&
                     connectionError(errs)) {
                     String msg = "TcpCommunicationSpi failed to establish connection to node, node will be dropped from " +
@@ -3384,7 +3463,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
                     else
                         U.warn(log, msg);
 
-                    getSpiContext().failNode(node.id(), "TcpCommunicationSpi failed to establish connection to node [" +
+                    ctx.failNode(node.id(), "TcpCommunicationSpi failed to establish connection to node [" +
                         "rmtNode=" + node +
                         ", errs=" + errs +
                         ", connectErrs=" + Arrays.toString(errs.getSuppressed()) + ']');
@@ -4590,77 +4669,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
     /**
      *
      */
-    private static class ConnectionKey {
-        /** */
-        private final UUID nodeId;
-
-        /** */
-        private final int idx;
-
-        /** */
-        private final long connCnt;
-
-        /**
-         * @param nodeId Node ID.
-         * @param idx Connection index.
-         * @param connCnt Connection counter (set only for incoming connections).
-         */
-        ConnectionKey(UUID nodeId, int idx, long connCnt) {
-            this.nodeId = nodeId;
-            this.idx = idx;
-            this.connCnt = connCnt;
-        }
-
-        /**
-         * @return Connection counter.
-         */
-        long connectCount() {
-            return connCnt;
-        }
-
-        /**
-         * @return Node ID.
-         */
-        UUID nodeId() {
-            return nodeId;
-        }
-
-        /**
-         * @return Connection index.
-         */
-        int connectionIndex() {
-            return idx;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            ConnectionKey key = (ConnectionKey) o;
-
-            return idx == key.idx && nodeId.equals(key.nodeId);
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            int res = nodeId.hashCode();
-            res = 31 * res + idx;
-            return res;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(ConnectionKey.class, this);
-        }
-    }
-
-    /**
-     *
-     */
     interface ConnectionPolicy {
         /**
          * @return Thread connection index.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/ConnectionKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/ConnectionKey.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/ConnectionKey.java
new file mode 100644
index 0000000..0559df7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/ConnectionKey.java
@@ -0,0 +1,117 @@
+/*
+ * 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.spi.communication.tcp.internal;
+
+import java.util.UUID;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Connection Key.
+ */
+public class ConnectionKey {
+    /** */
+    private final UUID nodeId;
+
+    /** */
+    private final int idx;
+
+    /** */
+    private final long connCnt;
+
+    /** */
+    private final boolean dummy;
+
+    /**
+     * Creates ConnectionKey with false value of dummy flag.
+     *
+     * @param nodeId Node ID. Should be not null.
+     * @param idx Connection index.
+     * @param connCnt Connection counter (set only for incoming connections).
+     */
+    public ConnectionKey(@NotNull UUID nodeId, int idx, long connCnt) {
+        this(nodeId, idx, connCnt, false);
+    }
+
+    /**
+     * @param nodeId Node ID. Should be not null.
+     * @param idx Connection index.
+     * @param connCnt Connection counter (set only for incoming connections).
+     * @param dummy Indicates that session with this ConnectionKey is temporary
+     *              (for now dummy sessions are used only for Communication Failure Resolving process).
+     */
+    public ConnectionKey(@NotNull UUID nodeId, int idx, long connCnt, boolean dummy) {
+        this.nodeId = nodeId;
+        this.idx = idx;
+        this.connCnt = connCnt;
+        this.dummy = dummy;
+    }
+
+    /**
+     * @return Connection counter.
+     */
+    public long connectCount() {
+        return connCnt;
+    }
+
+    /**
+     * @return Node ID.
+     */
+    public UUID nodeId() {
+        return nodeId;
+    }
+
+    /**
+     * @return Connection index.
+     */
+    public int connectionIndex() {
+        return idx;
+    }
+
+    /**
+     * @return {@code True} if this ConnectionKey is dummy and serves temporary session.
+     */
+    public boolean dummy() {
+        return dummy;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        ConnectionKey key = (ConnectionKey) o;
+
+        return idx == key.idx && nodeId.equals(key.nodeId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int res = nodeId.hashCode();
+        res = 31 * res + idx;
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(ConnectionKey.class, this);
+    }
+}