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

[13/13] ignite git commit: zk

zk


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

Branch: refs/heads/ignite-zk
Commit: d977dcb81eba54a746f829e2291a098330ddf45e
Parents: 1538ada
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 22 11:37:35 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 22 14:11:00 2017 +0300

----------------------------------------------------------------------
 .../spi/discovery/zk/ZookeeperDiscoverySpi.java |   4 +-
 .../discovery/zk/internal/ZkIgnitePaths.java    |  56 ++++-
 .../discovery/zk/internal/ZkRuntimeState.java   |   2 +-
 .../discovery/zk/internal/ZkTimeoutObject.java  |  51 +++++
 .../zk/internal/ZookeeperDiscoveryImpl.java     | 215 +++++++++++--------
 .../ZookeeperDiscoverySpiBasicTest.java         |  23 +-
 6 files changed, 252 insertions(+), 99 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d977dcb8/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java
index 559a1db..fdad5d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java
@@ -445,7 +445,7 @@ public class ZookeeperDiscoverySpi extends IgniteSpiAdapter implements Discovery
             internalLsnr);
 
         try {
-            impl.joinTopology();
+            impl.startJoinAndWait();
         }
         catch (InterruptedException e) {
             Thread.currentThread().interrupt();
@@ -519,7 +519,7 @@ public class ZookeeperDiscoverySpi extends IgniteSpiAdapter implements Discovery
     }
 
     /**
-     * Used for tests (call via reflection).
+     * Used in tests (called via reflection).
      *
      * @return Copy of SPI.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/d977dcb8/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkIgnitePaths.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkIgnitePaths.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkIgnitePaths.java
index ba3872c..c095e73 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkIgnitePaths.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkIgnitePaths.java
@@ -27,6 +27,9 @@ class ZkIgnitePaths {
     static final String PATH_SEPARATOR = "/";
 
     /** */
+    static final byte CLIENT_NODE_FLAG_MASK = 0x01;
+
+    /** */
     private static final int UUID_LEN = 36;
 
     /** Directory to store joined node data. */
@@ -181,6 +184,28 @@ class ZkIgnitePaths {
     }
 
     /**
+     * @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 aliveClientNode(String path) {
+        return (aliveFlags(path) & CLIENT_NODE_FLAG_MASK) != 0;
+    }
+
+    /**
      * @param path Alive node zk path.
      * @return Node ID.
      */
@@ -193,7 +218,7 @@ class ZkIgnitePaths {
      * @return Node ID.
      */
     static UUID aliveNodeId(String path) {
-        // <uuid prefix>:<node id>|<alive seq>
+        // <uuid prefix>:<node id>:<flags>|<alive seq>
         int startIdx = ZkIgnitePaths.UUID_LEN + 1;
 
         String idStr = path.substring(startIdx, startIdx + ZkIgnitePaths.UUID_LEN);
@@ -319,4 +344,33 @@ class ZkIgnitePaths {
     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/d977dcb8/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRuntimeState.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRuntimeState.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRuntimeState.java
index 9c000b3..7479c68 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRuntimeState.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRuntimeState.java
@@ -49,7 +49,7 @@ class ZkRuntimeState {
     int joinDataPartCnt;
 
     /** */
-    IgniteSpiTimeoutObject joinTimeoutObj;
+    IgniteSpiTimeoutObject joinErrTimeoutObj;
 
     /** */
     long gridStartTime;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d977dcb8/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkTimeoutObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkTimeoutObject.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkTimeoutObject.java
new file mode 100644
index 0000000..b2d2344
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkTimeoutObject.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;
+
+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;
+
+    /**
+     * @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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d977dcb8/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java
index 1c033c0..de89df7 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java
@@ -105,10 +105,10 @@ public class ZookeeperDiscoveryImpl {
     static final String IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD = "IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD";
 
     /** */
-    static final String IGNITE_ZOOKEEPER_DISCOVERY_MAX_EVTS = "IGNITE_ZOOKEEPER_DISCOVERY_MAX_EVTS";
+    static final String IGNITE_ZOOKEEPER_DISCOVERY_SPI_MAX_EVTS = "IGNITE_ZOOKEEPER_DISCOVERY_SPI_MAX_EVTS";
 
     /** */
-    private static final String IGNITE_ZOOKEEPER_DISCOVERY_EVTS_THROTTLE = "IGNITE_ZOOKEEPER_DISCOVERY_EVTS_THROTTLE";
+    private static final String IGNITE_ZOOKEEPER_DISCOVERY_SPI_EVTS_THROTTLE = "IGNITE_ZOOKEEPER_DISCOVERY_SPI_EVTS_THROTTLE";
 
     /** */
     final ZookeeperDiscoverySpi spi;
@@ -211,7 +211,7 @@ public class ZookeeperDiscoveryImpl {
         this.spi = spi;
         this.igniteInstanceName = igniteInstanceName;
         this.connectString = spi.getZkConnectionString();
-        this.sesTimeout = spi.getSessionTimeout();
+        this.sesTimeout = (int)spi.getSessionTimeout();
         this.log = log.getLogger(getClass());
         this.locNode = locNode;
         this.lsnr = lsnr;
@@ -462,7 +462,7 @@ public class ZookeeperDiscoveryImpl {
         try {
             locNode.onClientDisconnected(newId);
 
-            joinTopology0(true, rtState.joined);
+            joinTopology(true, rtState.joined);
         }
         catch (Exception e) {
             U.error(log, "Failed to reconnect: " + e, e);
@@ -649,10 +649,12 @@ public class ZookeeperDiscoveryImpl {
     }
 
     /**
+     * Starts join procedure and waits for {@link EventType#EVT_NODE_JOINED} event for local node.
+     *
      * @throws InterruptedException If interrupted.
      */
-    public void joinTopology() throws InterruptedException {
-        joinTopology0(false, false);
+    public void startJoinAndWait() throws InterruptedException {
+        joinTopology(false, false);
 
         for (;;) {
             try {
@@ -680,46 +682,62 @@ public class ZookeeperDiscoveryImpl {
      *    in this case (need produce EVT_CLIENT_NODE_RECONNECTED event).
      * @throws InterruptedException If interrupted.
      */
-    private void joinTopology0(boolean reconnect, boolean prevJoined) throws InterruptedException {
-        IgniteDiscoverySpiInternalListener internalLsnr = this.internalLsnr;
+    private void joinTopology(boolean reconnect, boolean prevJoined) throws InterruptedException {
+        if (!busyLock.enterBusy())
+            return;
 
-        if (internalLsnr != null)
-            internalLsnr.beforeJoin(locNode, log);
+        try {
+            IgniteDiscoverySpiInternalListener internalLsnr = this.internalLsnr;
 
-        if (locNode.isClient() && reconnect)
-            locNode.setAttributes(spi.getSpiContext().nodeAttributes());
+            if (internalLsnr != null)
+                internalLsnr.beforeJoin(locNode, log);
 
-        marshalCredentialsOnJoin(locNode);
+            if (locNode.isClient() && reconnect)
+                locNode.setAttributes(spi.getSpiContext().nodeAttributes());
 
-        rtState = new ZkRuntimeState(prevJoined);
+            marshalCredentialsOnJoin(locNode);
 
-        DiscoveryDataBag discoDataBag = new DiscoveryDataBag(locNode.id());
+            synchronized (stateMux) {
+                if (connState == ConnectionState.STOPPED)
+                    return;
 
-        exchange.collect(discoDataBag);
+                connState = ConnectionState.STARTED;
+            }
 
-        ZkJoiningNodeData joinData = new ZkJoiningNodeData(locNode, discoDataBag.joiningNodeData());
+            ZkRuntimeState rtState = this.rtState = new ZkRuntimeState(prevJoined);
 
-        byte[] joinDataBytes;
+            DiscoveryDataBag discoDataBag = new DiscoveryDataBag(locNode.id());
 
-        try {
-            joinDataBytes = marshalZip(joinData);
-        }
-        catch (Exception e) {
-            throw new IgniteSpiException("Failed to marshal joining node data", e);
-        }
+            exchange.collect(discoDataBag);
 
-        try {
-            rtState.zkClient = new ZookeeperClient(igniteInstanceName,
-                log,
-                connectString,
-                sesTimeout,
-                new ConnectionLossListener());
+            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, joinDataBytes);
         }
-        catch (Exception e) {
-            throw new IgniteSpiException("Failed to create Zookeeper client", e);
+        finally {
+            busyLock.leaveBusy();
         }
-
-        startJoin(joinDataBytes);
     }
 
     /**
@@ -879,10 +897,7 @@ public class ZookeeperDiscoveryImpl {
      * @param joinDataBytes Joining node data.
      * @throws InterruptedException If interrupted.
      */
-    private void startJoin(final byte[] joinDataBytes) throws InterruptedException {
-        if (!busyLock.enterBusy())
-            return;
-
+    private void startJoin(ZkRuntimeState rtState, final byte[] joinDataBytes) throws InterruptedException {
         try {
             long startTime = System.currentTimeMillis();
 
@@ -890,8 +905,6 @@ public class ZookeeperDiscoveryImpl {
 
             String prefix = UUID.randomUUID().toString();
 
-            final ZkRuntimeState rtState = this.rtState;
-
             rtState.init(new ZkWatcher(rtState), new AliveNodeDataWatcher(rtState));
 
             ZookeeperClient zkClient = rtState.zkClient;
@@ -923,7 +936,7 @@ public class ZookeeperDiscoveryImpl {
             rtState.locNodeZkPath = zkClient.createSequential(
                 prefix,
                 zkPaths.aliveNodesDir,
-                zkPaths.aliveNodesDir + "/" + prefix + ":" + locNode.id() + "|",
+                zkPaths.aliveNodePathForCreate(prefix, locNode),
                 null,
                 EPHEMERAL_SEQUENTIAL);
 
@@ -942,22 +955,29 @@ public class ZookeeperDiscoveryImpl {
             As a minor optimization do not start watch join data immediately, but only if do not receive
             join event after some timeout.
              */
-            rtState.joinTimeoutObj = new CheckJoinStateTimeoutObject(
-                joinDataPath,
-                rtState);
+            CheckJoinErrorWatcher joinErrorWatcher = new CheckJoinErrorWatcher(5000, joinDataPath, rtState);
 
-            zkClient.getChildrenAsync(zkPaths.aliveNodesDir, null, new CheckCoordinatorCallback(rtState));
+            rtState.joinErrTimeoutObj = joinErrorWatcher.timeoutObj;
+
+            if (!locNode.isClient())
+                zkClient.getChildrenAsync(zkPaths.aliveNodesDir, null, new CheckCoordinatorCallback(rtState));
 
             zkClient.getDataAsync(zkPaths.evtsPath, rtState.watcher, rtState.watcher);
 
-            spi.getSpiContext().addTimeoutObject(rtState.joinTimeoutObj);
+            spi.getSpiContext().addTimeoutObject(rtState.joinErrTimeoutObj);
+
+            if (locNode.isClient() && spi.getJoinTimeout() > 0) {
+                if (!rtState.prevJoined) {
+                    JoinTimeoutObject joinTimeoutObj = new JoinTimeoutObject(spi.getJoinTimeout());
+
+                    spi.getSpiContext().addTimeoutObject(joinTimeoutObj);
+                }
+            }
         }
         catch (IgniteCheckedException | ZookeeperClientFailedException e) {
             throw new IgniteSpiException("Failed to initialize Zookeeper nodes", e);
         }
         finally {
-            busyLock.leaveBusy();
-
             connStartLatch.countDown();
         }
     }
@@ -1053,48 +1073,67 @@ public class ZookeeperDiscoveryImpl {
     /**
      *
      */
-    private class CheckJoinStateTimeoutObject extends ZkAbstractWatcher
-        implements IgniteSpiTimeoutObject, AsyncCallback.DataCallback {
-        /** */
-        private final IgniteUuid id = IgniteUuid.randomUuid();
+    private class JoinTimeoutObject extends ZkTimeoutObject {
+        /**
+         * @param timeout Timeout.
+         */
+        JoinTimeoutObject(long timeout) {
+            super(timeout);
+        }
 
-        /** */
-        private final long endTime = System.currentTimeMillis() + 5000;
+        /** {@inheritDoc} */
+        @Override public void onTimeout() {
+            if (rtState.joined)
+                return;
+
+            runInWorkerThread(new Runnable() {
+                @Override public void run() {
+                    synchronized (stateMux) {
+                        if (connState == ConnectionState.STOPPED)
+                            return;
+
+                        connState = ConnectionState.STOPPED;
+                    }
+                }
+            });
+        }
+    }
 
+    /**
+     *
+     */
+    private class CheckJoinErrorWatcher extends ZkAbstractWatcher implements AsyncCallback.DataCallback {
         /** */
         private final String joinDataPath;
 
+        /** */
+        private ZkTimeoutObject timeoutObj;
+
         /**
-         * @param joinDataPath Node joined data path.
-         * @param rtState State.
+         * @param timeout Timeout.
+         * @param joinDataPath0 Node joined data path.
+         * @param rtState0 State.
          */
-        CheckJoinStateTimeoutObject(String joinDataPath, ZkRuntimeState rtState) {
-            super(rtState, ZookeeperDiscoveryImpl.this);
+        CheckJoinErrorWatcher(long timeout, String joinDataPath0, ZkRuntimeState rtState0) {
+            super(rtState0, ZookeeperDiscoveryImpl.this);
 
-            this.joinDataPath = joinDataPath;
-        }
+            this.joinDataPath = joinDataPath0;
 
-        /** {@inheritDoc} */
-        @Override public IgniteUuid id() {
-            return id;
-        }
-
-        /** {@inheritDoc} */
-        @Override public long endTime() {
-            return endTime;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onTimeout() {
-            if (rtState.errForClose != null || rtState.joined)
-                return;
+            timeoutObj = new ZkTimeoutObject(timeout) {
+                @Override public void onTimeout() {
+                    if (rtState.errForClose != null || rtState.joined)
+                        return;
 
-            synchronized (stateMux) {
-                if (connState != ConnectionState.STARTED)
-                    return;
-            }
+                    synchronized (stateMux) {
+                        if (connState != ConnectionState.STARTED)
+                            return;
+                    }
 
-            rtState.zkClient.getDataAsync(joinDataPath, this, this);
+                    rtState.zkClient.getDataAsync(joinDataPath,
+                        CheckJoinErrorWatcher.this,
+                        CheckJoinErrorWatcher.this);
+                }
+            };
         }
 
         /** {@inheritDoc} */
@@ -1338,7 +1377,7 @@ public class ZookeeperDiscoveryImpl {
 
         int newEvts = 0;
 
-        final int MAX_NEW_EVTS = IgniteSystemProperties.getInteger(IGNITE_ZOOKEEPER_DISCOVERY_MAX_EVTS, 100);
+        final int MAX_NEW_EVTS = IgniteSystemProperties.getInteger(IGNITE_ZOOKEEPER_DISCOVERY_SPI_MAX_EVTS, 100);
 
         List<ZookeeperClusterNode> failedNodes = null;
 
@@ -1419,7 +1458,7 @@ public class ZookeeperDiscoveryImpl {
      *
      */
     private void throttleNewEventsGeneration() {
-        long delay = IgniteSystemProperties.getLong(IGNITE_ZOOKEEPER_DISCOVERY_EVTS_THROTTLE, 0);
+        long delay = IgniteSystemProperties.getLong(IGNITE_ZOOKEEPER_DISCOVERY_SPI_EVTS_THROTTLE, 0);
 
         if (delay > 0) {
             if (log.isInfoEnabled())
@@ -1852,7 +1891,7 @@ public class ZookeeperDiscoveryImpl {
      */
     @SuppressWarnings("unchecked")
     private void newClusterStarted(int locInternalId) throws Exception {
-        spi.getSpiContext().removeTimeoutObject(rtState.joinTimeoutObj);
+        spi.getSpiContext().removeTimeoutObject(rtState.joinErrTimeoutObj);
 
         cleanupPreviousClusterData();
 
@@ -1875,11 +1914,6 @@ public class ZookeeperDiscoveryImpl {
 
         final List<ClusterNode> topSnapshot = Collections.singletonList((ClusterNode)locNode);
 
-        synchronized (stateMux) {
-            if (connState == ConnectionState.DISCONNECTED)
-                connState = ConnectionState.STARTED;
-        }
-
         lsnr.onDiscovery(EventType.EVT_NODE_JOINED,
             1L,
             locNode,
@@ -2354,7 +2388,7 @@ public class ZookeeperDiscoveryImpl {
         if (log.isInfoEnabled())
             log.info("Local join event data: " + evtData + ']');
 
-        spi.getSpiContext().removeTimeoutObject(rtState.joinTimeoutObj);
+        spi.getSpiContext().removeTimeoutObject(rtState.joinErrTimeoutObj);
 
         String path = zkPaths.joinEventDataPathForJoined(evtData.eventId());
 
@@ -2389,11 +2423,6 @@ public class ZookeeperDiscoveryImpl {
 
         final List<ClusterNode> topSnapshot = rtState.top.topologySnapshot();
 
-        synchronized (stateMux) {
-            if (connState == ConnectionState.DISCONNECTED)
-                connState = ConnectionState.STARTED;
-        }
-
         lsnr.onDiscovery(evtData.eventType(),
             evtData.topologyVersion(),
             locNode,

http://git-wip-us.apache.org/repos/asf/ignite/blob/d977dcb8/modules/core/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiBasicTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiBasicTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiBasicTest.java
index 57e53fc..524e69b 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiBasicTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiBasicTest.java
@@ -1216,13 +1216,13 @@ public class ZookeeperDiscoverySpiBasicTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testConcurrentStartStop2_EventsThrottle() throws Exception {
-        System.setProperty(ZookeeperDiscoveryImpl.IGNITE_ZOOKEEPER_DISCOVERY_MAX_EVTS, "1");
+        System.setProperty(ZookeeperDiscoveryImpl.IGNITE_ZOOKEEPER_DISCOVERY_SPI_MAX_EVTS, "1");
 
         try {
             concurrentStartStop(5);
         }
         finally {
-            System.clearProperty(ZookeeperDiscoveryImpl.IGNITE_ZOOKEEPER_DISCOVERY_MAX_EVTS);
+            System.clearProperty(ZookeeperDiscoveryImpl.IGNITE_ZOOKEEPER_DISCOVERY_SPI_MAX_EVTS);
         }
     }
 
@@ -2512,6 +2512,25 @@ public class ZookeeperDiscoverySpiBasicTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testClientStartNoServers() throws Exception {
+        IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                client = true;
+
+                startGrid(0);
+
+                return null;
+            }
+        });
+
+        waitSpi(getTestIgniteInstanceName(0));
+
+        fut.get();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testStartNoZk() throws Exception {
         stopZkCluster();