You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by iv...@apache.org on 2015/07/09 11:41:41 UTC

[2/2] incubator-ignite git commit: #ignite-1109: add consistentId to IgniteConfiguration.

#ignite-1109: add consistentId to IgniteConfiguration.


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

Branch: refs/heads/ignite-1109
Commit: 4f85586738a434b8fa6c23be81c0b2cd8f267703
Parents: 98b6487
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Jul 9 12:29:57 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Jul 9 12:29:57 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteCache.java     |  3 +-
 .../affinity/AffinityNodeHashResolver.java      |  6 ++
 .../rendezvous/RendezvousAffinityFunction.java  | 23 +++++-
 .../configuration/CacheConfiguration.java       |  3 +-
 .../configuration/IgniteConfiguration.java      | 26 +++++++
 .../apache/ignite/internal/IgniteKernal.java    |  2 +
 .../ignite/internal/IgniteNodeAttributes.java   |  3 +
 .../processors/cache/GridCacheProcessor.java    | 34 +++------
 .../processors/cache/IgniteInternalCache.java   |  3 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  3 +-
 .../tcp/internal/TcpDiscoveryNode.java          | 15 +++-
 ...DiscoveryNodeConfigConsistentIdSelfTest.java | 75 ++++++++++++++++++++
 .../IgniteSpiDiscoverySelfTestSuite.java        |  1 +
 13 files changed, 164 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4f855867/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index c8d6d7a..750c71d 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -34,6 +34,7 @@ import javax.cache.event.*;
 import javax.cache.expiry.*;
 import javax.cache.integration.*;
 import javax.cache.processor.*;
+import java.io.*;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.locks.*;
@@ -559,7 +560,7 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
      * the left nodes, and that nodes are restarted before
      * {@link CacheConfiguration#getRebalanceDelay() rebalanceDelay} expires. To place nodes
      * on the same place in consistent hash ring, use
-     * {@link RendezvousAffinityFunction#setHashIdResolver(AffinityNodeHashResolver)} to make sure that
+     * {@link IgniteConfiguration#setConsistentId(Serializable)} to make sure that
      * a node maps to the same hash ID if re-started.
      * <p>
      * See {@link CacheConfiguration#getRebalanceDelay()} for more information on how to configure

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4f855867/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityNodeHashResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityNodeHashResolver.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityNodeHashResolver.java
index d00b043..cad4b55 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityNodeHashResolver.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityNodeHashResolver.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.cache.affinity;
 
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
 
 import java.io.*;
 
@@ -31,7 +32,11 @@ import java.io.*;
  * Note that on case clients exist they will query this object from the server and use it for affinity calculation.
  * Therefore you must ensure that server and clients can marshal and unmarshal this object in portable format,
  * i.e. all parties have object class(es) configured as portable.
+ *
+ * @deprecated Use {@link IgniteConfiguration#setConsistentId(Serializable)}
+ * instead (refer to its JavaDoc for details).
  */
+@Deprecated
 public interface AffinityNodeHashResolver extends Serializable {
     /**
      * Resolve alternate hash value for the given Grid node.
@@ -39,5 +44,6 @@ public interface AffinityNodeHashResolver extends Serializable {
      * @param node Grid node.
      * @return Resolved hash ID.
      */
+    @Deprecated
     public Object resolve(ClusterNode node);
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4f855867/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
index 2b26630..862caa4 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
@@ -20,6 +20,7 @@ package org.apache.ignite.cache.affinity.rendezvous;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -88,7 +89,7 @@ public class RendezvousAffinityFunction implements AffinityFunction, Externaliza
     private IgniteBiPredicate<ClusterNode, ClusterNode> backupFilter;
 
     /** Hash ID resolver. */
-    private AffinityNodeHashResolver hashIdRslvr = new AffinityNodeAddressHashResolver();
+    private AffinityNodeHashResolver hashIdRslvr = null;
 
     /** Ignite instance. */
     @IgniteInstanceResource
@@ -204,6 +205,7 @@ public class RendezvousAffinityFunction implements AffinityFunction, Externaliza
      *
      * @return Hash ID resolver.
      */
+    @Deprecated
     public AffinityNodeHashResolver getHashIdResolver() {
         return hashIdRslvr;
     }
@@ -219,7 +221,11 @@ public class RendezvousAffinityFunction implements AffinityFunction, Externaliza
      * repartitioning.
      *
      * @param hashIdRslvr Hash ID resolver.
+     *
+     * @deprecated Use {@link IgniteConfiguration#setConsistentId(Serializable)}
+     * instead (refer to its JavaDoc for details).
      */
+    @Deprecated
     public void setHashIdResolver(AffinityNodeHashResolver hashIdRslvr) {
         this.hashIdRslvr = hashIdRslvr;
     }
@@ -273,6 +279,19 @@ public class RendezvousAffinityFunction implements AffinityFunction, Externaliza
     }
 
     /**
+     * Resolves node hash.
+     *
+     * @param node Cluster node;
+     * @return Node hash.
+     */
+    public Object resolveNodeHash(ClusterNode node) {
+        if (hashIdRslvr != null)
+            return hashIdRslvr.resolve(node);
+        else
+            return node.consistentId();
+    }
+
+    /**
      * Returns collection of nodes (primary first) for specified partition.
      */
     public List<ClusterNode> assignPartition(int part, List<ClusterNode> nodes, int backups,
@@ -285,7 +304,7 @@ public class RendezvousAffinityFunction implements AffinityFunction, Externaliza
         MessageDigest d = digest.get();
 
         for (ClusterNode node : nodes) {
-            Object nodeHash = hashIdRslvr.resolve(node);
+            Object nodeHash = resolveNodeHash(node);
 
             try {
                 ByteArrayOutputStream out = new ByteArrayOutputStream();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4f855867/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 63c7269..183b673 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -33,6 +33,7 @@ import org.jetbrains.annotations.*;
 import javax.cache.*;
 import javax.cache.configuration.*;
 import javax.cache.expiry.*;
+import java.io.*;
 import java.util.*;
 
 /**
@@ -1310,7 +1311,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * For better efficiency user should usually make sure that new nodes get placed on
      * the same place of consistent hash ring as the left nodes, and that nodes are
      * restarted before this delay expires. To place nodes on the same place in consistent hash ring,
-     * use {@link RendezvousAffinityFunction#setHashIdResolver(AffinityNodeHashResolver)}
+     * use {@link IgniteConfiguration#setConsistentId(Serializable)}
      * to make sure that a node maps to the same hash ID event if restarted. As an example,
      * node IP address and port combination may be used in this case.
      * <p>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4f855867/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 2d36c7a..34995e5 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
@@ -59,6 +59,7 @@ import javax.cache.expiry.*;
 import javax.cache.integration.*;
 import javax.cache.processor.*;
 import javax.management.*;
+import java.io.*;
 import java.lang.management.*;
 import java.util.*;
 
@@ -400,6 +401,9 @@ public class IgniteConfiguration {
     /** Cache store session listeners. */
     private Factory<CacheStoreSessionListener>[] storeSesLsnrs;
 
+    /** Consistent globally unique node ID which survives node restarts. */
+    private Serializable consistentId;
+
     /**
      * Creates valid grid configuration with all default values.
      */
@@ -491,6 +495,7 @@ public class IgniteConfiguration {
         utilityCachePoolSize = cfg.getUtilityCacheThreadPoolSize();
         waitForSegOnStart = cfg.isWaitForSegmentOnStart();
         warmupClos = cfg.getWarmupClosure();
+        consistentId = cfg.getConsistentId();
     }
 
     /**
@@ -558,6 +563,27 @@ public class IgniteConfiguration {
     }
 
     /**
+     * Sets consistent globally unique node ID which survives node restarts.
+     *
+     * @param consistentId Node consistent id.
+     * @return {@code this} for chaining.
+     */
+    public IgniteConfiguration setConsistentId(Serializable consistentId) {
+        this.consistentId = consistentId;
+
+        return this;
+    }
+
+    /**
+     * Gets consistent globally unique node ID which survives node restarts.
+     *
+     * @return Node consistent id.
+     */
+    public Serializable getConsistentId() {
+        return this.consistentId;
+    }
+
+    /**
      * Should return any user-defined attributes to be added to this node. These attributes can
      * then be accessed on nodes by calling {@link ClusterNode#attribute(String)} or
      * {@link ClusterNode#attributes()} methods.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4f855867/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 d6ddf79..086fa53 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
@@ -1167,6 +1167,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
         add(ATTR_CONSISTENCY_CHECK_SKIPPED, getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK));
 
+        add(ATTR_NODE_CONSISTENT_ID, cfg.getConsistentId());
+
         // Build a string from JVM arguments, because parameters with spaces are split.
         SB jvmArgs = new SB(512);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4f855867/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
index 928db5e..10b8df0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
@@ -132,6 +132,9 @@ public final class IgniteNodeAttributes {
     /** Configuration consistency check disabled flag. */
     public static final String ATTR_CONSISTENCY_CHECK_SKIPPED = ATTR_PREFIX + ".consistency.check.skipped";
 
+    /** Node consistent id. */
+    public static final String ATTR_NODE_CONSISTENT_ID = ATTR_PREFIX + ".consistent.id";
+
     /**
      * Enforces singleton.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4f855867/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 de1eac2..485e060 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
@@ -159,15 +159,11 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             if (cfg.getCacheMode() == PARTITIONED) {
                 RendezvousAffinityFunction aff = new RendezvousAffinityFunction();
 
-                aff.setHashIdResolver(new AffinityNodeAddressHashResolver());
-
                 cfg.setAffinity(aff);
             }
             else if (cfg.getCacheMode() == REPLICATED) {
                 RendezvousAffinityFunction aff = new RendezvousAffinityFunction(false, 512);
 
-                aff.setHashIdResolver(new AffinityNodeAddressHashResolver());
-
                 cfg.setAffinity(aff);
 
                 cfg.setBackups(Integer.MAX_VALUE);
@@ -175,16 +171,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             else
                 cfg.setAffinity(new LocalAffinityFunction());
         }
-        else {
-            if (cfg.getCacheMode() == PARTITIONED) {
-                if (cfg.getAffinity() instanceof RendezvousAffinityFunction) {
-                    RendezvousAffinityFunction aff = (RendezvousAffinityFunction)cfg.getAffinity();
-
-                    if (aff.getHashIdResolver() == null)
-                        aff.setHashIdResolver(new AffinityNodeAddressHashResolver());
-                }
-            }
-        }
 
         if (cfg.getCacheMode() == REPLICATED)
             cfg.setBackups(Integer.MAX_VALUE);
@@ -2257,25 +2243,25 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 if (cfg.getAffinity() instanceof RendezvousAffinityFunction) {
                     RendezvousAffinityFunction aff = (RendezvousAffinityFunction)cfg.getAffinity();
 
-                    AffinityNodeHashResolver hashIdRslvr = aff.getHashIdResolver();
-
-                    assert hashIdRslvr != null;
-
-                    Object nodeHashObj = hashIdRslvr.resolve(node);
+                    Object nodeHashObj = aff.resolveNodeHash(node);
 
                     for (ClusterNode topNode : ctx.discovery().allNodes()) {
-                        Object topNodeHashObj = hashIdRslvr.resolve(topNode);
+                        Object topNodeHashObj = aff.resolveNodeHash(topNode);
 
                         if (nodeHashObj.hashCode() == topNodeHashObj.hashCode()) {
+                            String hashIdRslvrName = "";
+
+                            if (aff.getHashIdResolver() != null)
+                                hashIdRslvrName = ", hashIdResolverClass=" +
+                                    aff.getHashIdResolver().getClass().getName();
+
                             String errMsg = "Failed to add node to topology because it has the same hash code for " +
                                 "partitioned affinity as one of existing nodes [cacheName=" + U.maskName(cfg.getName()) +
-                                ", hashIdResolverClass=" + hashIdRslvr.getClass().getName() +
-                                ", existingNodeId=" + topNode.id() + ']';
+                                hashIdRslvrName + ", existingNodeId=" + topNode.id() + ']';
 
                             String sndMsg = "Failed to add node to topology because it has the same hash code for " +
                                 "partitioned affinity as one of existing nodes [cacheName=" + U.maskName(cfg.getName()) +
-                                ", hashIdResolverClass=" + hashIdRslvr.getClass().getName() + ", existingNodeId=" +
-                                topNode.id() + ']';
+                                hashIdRslvrName + ", existingNodeId=" + topNode.id() + ']';
 
                             return new IgniteNodeValidationResult(topNode.id(), errMsg, sndMsg);
                         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4f855867/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
index 9972f92..5d8d7fa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
@@ -34,6 +34,7 @@ import org.jetbrains.annotations.*;
 import javax.cache.*;
 import javax.cache.expiry.*;
 import javax.cache.processor.*;
+import java.io.*;
 import java.sql.*;
 import java.util.*;
 import java.util.Date;
@@ -1456,7 +1457,7 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K, V>> {
      * the left nodes, and that nodes are restarted before
      * {@link CacheConfiguration#getRebalanceDelay() rebalanceDelay} expires. To place nodes
      * on the same place in consistent hash ring, use
-     * {@link org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction#setHashIdResolver(AffinityNodeHashResolver)} to make sure that
+     * {@link IgniteConfiguration#setConsistentId(Serializable)} to make sure that
      * a node maps to the same hash ID if re-started.
      * <p>
      * See {@link org.apache.ignite.configuration.CacheConfiguration#getRebalanceDelay()} for more information on how to configure

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4f855867/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 7663fe6..1e2c365 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
@@ -849,7 +849,8 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
             addrs.get2(),
             srvPort,
             metricsProvider,
-            locNodeVer);
+            locNodeVer,
+            ignite.configuration().getConsistentId());
 
         if (addExtAddrAttr) {
             Collection<InetSocketAddress> extAddrs = addrRslvr == null ? null :

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4f855867/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 ed954af..065b3c0 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
@@ -131,11 +131,12 @@ public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements Cluste
      * @param discPort Port.
      * @param metricsProvider Metrics provider.
      * @param ver Version.
+     * @param consistentId Node consistent id.
      */
     public TcpDiscoveryNode(UUID id,
         Collection<String> addrs,
         Collection<String> hostNames, int discPort,
-        DiscoveryMetricsProvider metricsProvider, IgniteProductVersion ver)
+        DiscoveryMetricsProvider metricsProvider, IgniteProductVersion ver, Serializable consistentId)
     {
         assert id != null;
         assert !F.isEmpty(addrs);
@@ -152,7 +153,12 @@ public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements Cluste
         this.discPort = discPort;
         this.metricsProvider = metricsProvider;
         this.ver = ver;
-        consistentId = U.consistentId(sortedAddrs, discPort);
+
+        if (consistentId != null)
+            this.consistentId = consistentId;
+        else
+            this.consistentId = U.consistentId(sortedAddrs, discPort);
+
         metrics = metricsProvider.metrics();
         cacheMetrics = metricsProvider.cacheMetrics();
         sockAddrs = U.toSocketAddresses(this, discPort);
@@ -501,7 +507,10 @@ public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements Cluste
 
         sockAddrs = U.toSocketAddresses(this, discPort);
 
-        consistentId = U.consistentId(addrs, discPort);
+        if (attrs.containsKey(ATTR_NODE_CONSISTENT_ID))
+            consistentId = attrs.get(ATTR_NODE_CONSISTENT_ID);
+        else
+            consistentId = U.consistentId(addrs, discPort);
 
         // Cluster metrics
         byte[] mtr = U.readByteArray(in);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4f855867/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryNodeConfigConsistentIdSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryNodeConfigConsistentIdSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryNodeConfigConsistentIdSelfTest.java
new file mode 100644
index 0000000..4610a2d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryNodeConfigConsistentIdSelfTest.java
@@ -0,0 +1,75 @@
+/*
+ * 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.tcp;
+
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.io.*;
+
+/**
+ * Test for {@link IgniteConfiguration#consistentId}.
+ */
+public class TcpDiscoveryNodeConfigConsistentIdSelfTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setLocalHost("0.0.0.0");
+
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(ipFinder));
+
+        cfg.setConsistentId(gridName);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        startGrids(1);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConsistentId() throws Exception {
+        Object id0 = grid(0).localNode().consistentId();
+        Serializable id1 = grid(0).configuration().getConsistentId();
+
+        assertEquals(id0, id1);
+        assertEquals(grid(0).name(), id0);
+
+        for (int i = 0; i < 10; ++i) {
+            stopAllGrids();
+
+            startGrids(1);
+
+            assertEquals(id0, grid(0).localNode().consistentId());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4f855867/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
index 6f59f14..b7014ad 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
@@ -56,6 +56,7 @@ public class IgniteSpiDiscoverySelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(TcpClientDiscoverySpiMulticastTest.class));
 
         suite.addTest(new TestSuite(TcpDiscoveryNodeConsistentIdSelfTest.class));
+        suite.addTest(new TestSuite(TcpDiscoveryNodeConfigConsistentIdSelfTest.class));
 
         suite.addTest(new TestSuite(TcpDiscoveryRestartTest.class));
         suite.addTest(new TestSuite(TcpDiscoveryMultiThreadedTest.class));