Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 34B3818FB6 for ; Thu, 28 Jan 2016 09:35:19 +0000 (UTC) Received: (qmail 44228 invoked by uid 500); 28 Jan 2016 09:28:36 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 44193 invoked by uid 500); 28 Jan 2016 09:28:36 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 44184 invoked by uid 99); 28 Jan 2016 09:28:36 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 28 Jan 2016 09:28:36 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 05F75DFFF4; Thu, 28 Jan 2016 09:28:36 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: agoncharuk@apache.org To: commits@ignite.apache.org Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: ignite git commit: IGNITE-1355 - Fixed potential NPE in CacheAffinityProxy - Fixes #263. Date: Thu, 28 Jan 2016 09:28:36 +0000 (UTC) Repository: ignite Updated Branches: refs/heads/master a34d7058b -> 550a4ea7e IGNITE-1355 - Fixed potential NPE in CacheAffinityProxy - Fixes #263. Signed-off-by: Alexey Goncharuk Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/550a4ea7 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/550a4ea7 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/550a4ea7 Branch: refs/heads/master Commit: 550a4ea7eab41a3d27ff29d1e3a8df09d698524a Parents: a34d705 Author: ashutak Authored: Thu Jan 28 12:20:21 2016 +0300 Committer: Alexey Goncharuk Committed: Thu Jan 28 12:20:21 2016 +0300 ---------------------------------------------------------------------- .../examples/datagrid/CacheAffinityExample.java | 8 +- .../java8/datagrid/CacheAffinityExample.java | 6 +- .../internal/client/ClientGetAffinityTask.java | 4 +- .../java/org/apache/ignite/IgniteCluster.java | 7 +- .../apache/ignite/cache/affinity/Affinity.java | 24 +- .../affinity/GridAffinityProcessor.java | 60 ++-- .../cache/GridCacheAffinityManager.java | 47 ++- .../cache/affinity/GridCacheAffinityImpl.java | 48 ++- .../ignite/internal/GridAffinityMappedTest.java | 8 +- .../internal/GridAffinityNoCacheSelfTest.java | 290 +++++++++++++++++++ .../internal/GridAffinityP2PSelfTest.java | 8 +- .../ignite/internal/GridAffinitySelfTest.java | 8 +- .../cache/GridCacheAbstractFullApiSelfTest.java | 4 +- .../cache/GridCacheAffinityRoutingSelfTest.java | 10 +- .../GridCacheConcurrentTxMultiNodeTest.java | 4 +- .../GridCacheDaemonNodeAbstractSelfTest.java | 17 +- .../cache/GridCacheDeploymentSelfTest.java | 8 +- .../cache/GridCacheEntryMemorySizeSelfTest.java | 6 +- ...hePartitionedProjectionAffinitySelfTest.java | 8 +- .../cache/GridCachePutAllFailoverSelfTest.java | 4 +- .../dht/GridCacheDhtMultiBackupTest.java | 4 +- .../near/GridCacheNearOnlyTopologySelfTest.java | 4 +- .../near/GridCacheNearTxMultiNodeSelfTest.java | 4 +- ...titionedExplicitLockNodeFailureSelfTest.java | 6 +- ...ridCacheContinuousQueryAbstractSelfTest.java | 10 +- .../processors/igfs/IgfsStreamsSelfTest.java | 4 +- .../ignite/loadtests/dsi/GridDsiClient.java | 4 +- .../tcp/GridCacheDhtLockBackupSelfTest.java | 4 +- .../testsuites/IgniteComputeGridTestSuite.java | 4 +- 29 files changed, 499 insertions(+), 124 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheAffinityExample.java ---------------------------------------------------------------------- diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheAffinityExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheAffinityExample.java index 2ec0620..c059ced 100644 --- a/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheAffinityExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheAffinityExample.java @@ -22,10 +22,10 @@ import java.util.Collection; import java.util.Map; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteCluster; import org.apache.ignite.IgniteCompute; import org.apache.ignite.IgniteException; import org.apache.ignite.Ignition; +import org.apache.ignite.cache.affinity.Affinity; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.examples.ExampleNodeStartup; import org.apache.ignite.lang.IgniteRunnable; @@ -103,7 +103,7 @@ public final class CacheAffinityExample { } /** - * Collocates jobs with keys they need to work on using {@link IgniteCluster#mapKeysToNodes(String, Collection)} + * Collocates jobs with keys they need to work on using {@link Affinity#mapKeysToNodes(Collection)} * method. The difference from {@code affinityRun(...)} method is that here we process multiple keys * in a single job. */ @@ -116,7 +116,7 @@ public final class CacheAffinityExample { keys.add(i); // Map all keys to nodes. - Map> mappings = ignite.cluster().mapKeysToNodes(CACHE_NAME, keys); + Map> mappings = ignite.affinity(CACHE_NAME).mapKeysToNodes(keys); for (Map.Entry> mapping : mappings.entrySet()) { ClusterNode node = mapping.getKey(); @@ -139,4 +139,4 @@ public final class CacheAffinityExample { } } } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/examples/src/main/java8/org/apache/ignite/examples/java8/datagrid/CacheAffinityExample.java ---------------------------------------------------------------------- diff --git a/examples/src/main/java8/org/apache/ignite/examples/java8/datagrid/CacheAffinityExample.java b/examples/src/main/java8/org/apache/ignite/examples/java8/datagrid/CacheAffinityExample.java index fbf348f..2867bf1 100644 --- a/examples/src/main/java8/org/apache/ignite/examples/java8/datagrid/CacheAffinityExample.java +++ b/examples/src/main/java8/org/apache/ignite/examples/java8/datagrid/CacheAffinityExample.java @@ -22,11 +22,11 @@ import java.util.Collection; import java.util.Map; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteCluster; import org.apache.ignite.IgniteCompute; import org.apache.ignite.IgniteException; import org.apache.ignite.Ignition; import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.affinity.Affinity; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.examples.ExampleNodeStartup; @@ -105,7 +105,7 @@ public final class CacheAffinityExample { } /** - * Collocates jobs with keys they need to work on using {@link IgniteCluster#mapKeysToNodes(String, Collection)} + * Collocates jobs with keys they need to work on using {@link Affinity#mapKeysToNodes(Collection)} * method. The difference from {@code affinityRun(...)} method is that here we process multiple keys * in a single job. */ @@ -118,7 +118,7 @@ public final class CacheAffinityExample { keys.add(i); // Map all keys to nodes. - Map> mappings = ignite.cluster().mapKeysToNodes(CACHE_NAME, keys); + Map> mappings = ignite.affinity(CACHE_NAME).mapKeysToNodes(keys); for (Map.Entry> mapping : mappings.entrySet()) { ClusterNode node = mapping.getKey(); http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientGetAffinityTask.java ---------------------------------------------------------------------- diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientGetAffinityTask.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientGetAffinityTask.java index 94367d1..509324a 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientGetAffinityTask.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientGetAffinityTask.java @@ -50,7 +50,7 @@ public class ClientGetAffinityTask extends TaskSingleJobSplitAdapter Map> mapKeysToNodes(@Nullable String cacheName, @Nullable Collection keys) throws IgniteException; @@ -148,7 +151,9 @@ public interface IgniteCluster extends ClusterGroup, IgniteAsyncSupport { * @return Primary node for the key or {@code null} if cache with given name * is not present in the grid. * @throws IgniteException If failed to map key. + * @deprecated Use {@link Affinity#mapKeyToNode(Object)} instead. */ + @Deprecated public ClusterNode mapKeyToNode(@Nullable String cacheName, K key) throws IgniteException; /** @@ -343,4 +348,4 @@ public interface IgniteCluster extends ClusterGroup, IgniteAsyncSupport { /** {@inheritDoc} */ @Override public IgniteCluster withAsync(); -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/main/java/org/apache/ignite/cache/affinity/Affinity.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/Affinity.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/Affinity.java index 28dc0ec..a5756e9 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/Affinity.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/Affinity.java @@ -19,13 +19,14 @@ package org.apache.ignite.cache.affinity; import java.util.Collection; import java.util.Map; +import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; import org.jetbrains.annotations.Nullable; /** * Provides affinity information to detect which node is primary and which nodes are * backups for a partitioned cache. You can get an instance of this interface by calling - * {@code Cache.affinity()} method. + * {@code Ignite.affinity(cacheName)} method. *

* Mapping of a key to a node is a three-step operation. First step will get an affinity key for given key * using {@link AffinityKeyMapper}. If mapper is not specified, the original key will be used. Second step @@ -41,6 +42,7 @@ public interface Affinity { * Gets number of partitions in cache according to configured affinity function. * * @return Number of cache partitions. + * @throws IgniteException If there are no alive nodes for this cache. * @see AffinityFunction * @see org.apache.ignite.configuration.CacheConfiguration#getAffinity() * @see org.apache.ignite.configuration.CacheConfiguration#setAffinity(AffinityFunction) @@ -52,6 +54,7 @@ public interface Affinity { * * @param key Key to get partition id for. * @return Partition id. + * @throws IgniteException If there are no alive nodes for this cache. * @see AffinityFunction * @see org.apache.ignite.configuration.CacheConfiguration#getAffinity() * @see org.apache.ignite.configuration.CacheConfiguration#setAffinity(AffinityFunction) @@ -64,6 +67,7 @@ public interface Affinity { * @param n Node to check. * @param key Key to check. * @return {@code True} if local node is the primary node for given key. + * @throws IgniteException If there are no alive nodes for this cache. */ public boolean isPrimary(ClusterNode n, K key); @@ -73,6 +77,7 @@ public interface Affinity { * @param n Node to check. * @param key Key to check. * @return {@code True} if local node is one of the backup nodes for given key. + * @throws IgniteException If there are no alive nodes for this cache. */ public boolean isBackup(ClusterNode n, K key); @@ -86,6 +91,7 @@ public interface Affinity { * @param n Node to check. * @param key Key to check. * @return {@code True} if local node is primary or backup for given key. + * @throws IgniteException If there are no alive nodes for this cache. */ public boolean isPrimaryOrBackup(ClusterNode n, K key); @@ -94,6 +100,7 @@ public interface Affinity { * * @param n Cluster node. * @return Partition ids for which given cluster node has primary ownership. + * @throws IgniteException If there are no alive nodes for this cache. * @see AffinityFunction * @see org.apache.ignite.configuration.CacheConfiguration#getAffinity() * @see org.apache.ignite.configuration.CacheConfiguration#setAffinity(AffinityFunction) @@ -105,6 +112,7 @@ public interface Affinity { * * @param n Cluster node. * @return Partition ids for which given cluster node has backup ownership. + * @throws IgniteException If there are no alive nodes for this cache. * @see AffinityFunction * @see org.apache.ignite.configuration.CacheConfiguration#getAffinity() * @see org.apache.ignite.configuration.CacheConfiguration#setAffinity(AffinityFunction) @@ -117,6 +125,7 @@ public interface Affinity { * * @param n Cluster node. * @return Partition ids for which given cluster node has any ownership, primary or backup. + * @throws IgniteException If there are no alive nodes for this cache. * @see AffinityFunction * @see org.apache.ignite.configuration.CacheConfiguration#getAffinity() * @see org.apache.ignite.configuration.CacheConfiguration#setAffinity(AffinityFunction) @@ -132,6 +141,7 @@ public interface Affinity { * @param key Key to map. * @return Key to be used for node-to-affinity mapping (may be the same * key as passed in). + * @throws IgniteException If there are no alive nodes for this cache. */ public Object affinityKey(K key); @@ -151,7 +161,8 @@ public interface Affinity { * * * @param keys Keys to map to nodes. - * @return Map of nodes to keys or empty map if there are no alive nodes for this cache. + * @return Map of nodes to keys. + * @throws IgniteException If there are no alive nodes for this cache. */ public Map> mapKeysToNodes(Collection keys); @@ -171,7 +182,8 @@ public interface Affinity { * * * @param key Keys to map to a node. - * @return Primary node for the key or {@code null} if there are no alive nodes for this cache. + * @return Primary node for the key. + * @throws IgniteException If there are no alive nodes for this cache. */ @Nullable public ClusterNode mapKeyToNode(K key); @@ -182,6 +194,7 @@ public interface Affinity { * @param key Key to get affinity nodes for. * @return Collection of primary and backup nodes for the key with primary node * always first. + * @throws IgniteException If there are no alive nodes for this cache. */ public Collection mapKeyToPrimaryAndBackups(K key); @@ -190,6 +203,7 @@ public interface Affinity { * * @param part Partition id. * @return Primary node for the given partition. + * @throws IgniteException If there are no alive nodes for this cache. * @see AffinityFunction * @see org.apache.ignite.configuration.CacheConfiguration#getAffinity() * @see org.apache.ignite.configuration.CacheConfiguration#setAffinity(AffinityFunction) @@ -201,6 +215,7 @@ public interface Affinity { * * @param parts Partition ids. * @return Mapping of given partitions to their primary nodes. + * @throws IgniteException If there are no alive nodes for this cache. * @see AffinityFunction * @see org.apache.ignite.configuration.CacheConfiguration#getAffinity() * @see org.apache.ignite.configuration.CacheConfiguration#setAffinity(AffinityFunction) @@ -214,6 +229,7 @@ public interface Affinity { * @param part Partition to get affinity nodes for. * @return Collection of primary and backup nodes for partition with primary node * always first. + * @throws IgniteException If there are no alive nodes for this cache. */ public Collection mapPartitionToPrimaryAndBackups(int part); -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java index 90306b0..8a0194c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java @@ -178,7 +178,8 @@ public class GridAffinityProcessor extends GridProcessorAdapter { * @return Picked node. * @throws IgniteCheckedException If failed. */ - @Nullable public ClusterNode mapKeyToNode(@Nullable String cacheName, K key, AffinityTopologyVersion topVer) throws IgniteCheckedException { + @Nullable public ClusterNode mapKeyToNode(@Nullable String cacheName, K key, + AffinityTopologyVersion topVer) throws IgniteCheckedException { Map> map = keysToNodes(cacheName, F.asList(key), topVer); return map != null ? F.first(map.keySet()) : null; @@ -209,20 +210,6 @@ public class GridAffinityProcessor extends GridProcessorAdapter { } /** - * Map single key to primary and backup nodes. - * - * @param cacheName Cache name. - * @param key Key to map. - * @return Affinity nodes, primary first. - * @throws IgniteCheckedException If failed. - */ - public List mapKeyToPrimaryAndBackups(@Nullable String cacheName, K key) - throws IgniteCheckedException - { - return mapKeyToPrimaryAndBackups(cacheName, key, ctx.discovery().topologyVersionEx()); - } - - /** * Gets affinity key for cache key. * * @param cacheName Cache name. @@ -318,7 +305,7 @@ public class GridAffinityProcessor extends GridProcessorAdapter { if (cache == null) return null; - GridCacheContext cctx = cache.context(); + GridCacheContext cctx = cache.context(); cctx.awaitStarted(); @@ -424,8 +411,7 @@ public class GridAffinityProcessor extends GridProcessorAdapter { } /** - * Requests {@link AffinityFunction} and - * {@link AffinityKeyMapper} from remote node. + * Requests {@link AffinityFunction} and {@link AffinityKeyMapper} from remote node. * * @param cacheName Name of cache on which affinity is requested. * @param topVer Topology version. @@ -518,7 +504,7 @@ public class GridAffinityProcessor extends GridProcessorAdapter { */ private List primaryAndBackups(AffinityInfo aff, K key) { if (key instanceof CacheObject && !(key instanceof BinaryObject)) - key = ((CacheObject) key).value(aff.cacheObjCtx, false); + key = ((CacheObject)key).value(aff.cacheObjCtx, false); int part = aff.affFunc.partition(aff.mapper.affinityKey(key)); @@ -801,7 +787,12 @@ public class GridAffinityProcessor extends GridProcessorAdapter { ctx.gateway().readLock(); try { - return GridAffinityProcessor.this.mapKeysToNodes(cacheName, keys); + if (F.isEmpty(keys)) + return Collections.emptyMap(); + + AffinityInfo affInfo = cache(); + + return affinityMap(affInfo, keys); } catch (IgniteCheckedException e) { throw new IgniteException(e); @@ -813,10 +804,16 @@ public class GridAffinityProcessor extends GridProcessorAdapter { /** {@inheritDoc} */ @Nullable @Override public ClusterNode mapKeyToNode(K key) { + A.notNull(key, "key"); + ctx.gateway().readLock(); try { - return GridAffinityProcessor.this.mapKeyToNode(cacheName, key); + AffinityInfo affInfo = cache(); + + Map> map = affinityMap(affInfo, Collections.singletonList(key)); + + return F.first(map.keySet()); } catch (IgniteCheckedException e) { throw new IgniteException(e); @@ -880,9 +877,7 @@ public class GridAffinityProcessor extends GridProcessorAdapter { ctx.gateway().readLock(); try { - AffinityInfo cache = cache(); - - return cache != null ? cache.assignment().get(part) : Collections.emptyList(); + return cache().assignment().get(part); } catch (IgniteCheckedException e) { throw new IgniteException(e); @@ -896,15 +891,14 @@ public class GridAffinityProcessor extends GridProcessorAdapter { * @return Affinity info for current topology version. * @throws IgniteCheckedException If failed. */ - @Nullable private AffinityInfo cache() throws IgniteCheckedException { - return affinityCache(cacheName, new AffinityTopologyVersion(topologyVersion())); - } + private AffinityInfo cache() throws IgniteCheckedException { + AffinityInfo aff = affinityCache(cacheName, ctx.discovery().topologyVersionEx()); - /** - * @return Topology version. - */ - private long topologyVersion() { - return ctx.discovery().topologyVersion(); + if (aff == null) + throw new IgniteException("Failed to find cache (cache was not started " + + "yet or cache was already stopped): " + cacheName); + + return aff; } } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java index eddffea..375219a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.Set; import java.util.UUID; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.events.DiscoveryEvent; import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException; @@ -42,6 +43,10 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter { /** */ private static final AffinityTopologyVersion TOP_FIRST = new AffinityTopologyVersion(1); + /** */ + public static final String FAILED_TO_FIND_CACHE_ERR_MSG = "Failed to find cache (cache was not started " + + "yet or cache was already stopped): "; + /** Affinity cached function. */ private GridAffinityAssignmentCache aff; @@ -189,7 +194,12 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter { * @return Partition count. */ public int partitions() { - return aff.partitions(); + GridAffinityAssignmentCache aff0 = aff; + + if (aff0 == null) + throw new IgniteException(FAILED_TO_FIND_CACHE_ERR_MSG + cctx.name()); + + return aff0.partitions(); } /** @@ -201,7 +211,12 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter { * @return Partition. */ public int partition(Object key) { - return aff.partition(key); + GridAffinityAssignmentCache aff0 = aff; + + if (aff0 == null) + throw new IgniteException(FAILED_TO_FIND_CACHE_ERR_MSG + cctx.name()); + + return aff0.partition(key); } /** @@ -222,7 +237,12 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter { if (cctx.isLocal()) topVer = new AffinityTopologyVersion(1); - return aff.nodes(part, topVer); + GridAffinityAssignmentCache aff0 = aff; + + if (aff0 == null) + throw new IgniteException(FAILED_TO_FIND_CACHE_ERR_MSG + cctx.name()); + + return aff0.nodes(part, topVer); } /** @@ -363,7 +383,12 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter { if (cctx.isLocal()) topVer = new AffinityTopologyVersion(1); - return aff.primaryPartitions(nodeId, topVer); + GridAffinityAssignmentCache aff0 = aff; + + if (aff0 == null) + throw new IgniteException(FAILED_TO_FIND_CACHE_ERR_MSG + cctx.name()); + + return aff0.primaryPartitions(nodeId, topVer); } /** @@ -375,14 +400,24 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter { if (cctx.isLocal()) topVer = new AffinityTopologyVersion(1); - return aff.backupPartitions(nodeId, topVer); + GridAffinityAssignmentCache aff0 = aff; + + if (aff0 == null) + throw new IgniteException(FAILED_TO_FIND_CACHE_ERR_MSG + cctx.name()); + + return aff0.backupPartitions(nodeId, topVer); } /** * @return Affinity-ready topology version. */ public AffinityTopologyVersion affinityTopologyVersion() { - return aff.lastVersion(); + GridAffinityAssignmentCache aff0 = aff; + + if (aff0 == null) + throw new IgniteException(FAILED_TO_FIND_CACHE_ERR_MSG + cctx.name()); + + return aff0.lastVersion(); } /** http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java index b9045e9..3bc71fe 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java @@ -23,12 +23,15 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; +import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.cache.affinity.Affinity; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.CacheObjectContext; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.A; @@ -39,6 +42,10 @@ import org.jetbrains.annotations.Nullable; * Affinity interface implementation. */ public class GridCacheAffinityImpl implements Affinity { + /** */ + public static final String FAILED_TO_FIND_CACHE_ERR_MSG = "Failed to find cache (cache was not started " + + "yet or cache was already stopped): "; + /** Cache context. */ private GridCacheContext cctx; @@ -56,7 +63,12 @@ public class GridCacheAffinityImpl implements Affinity { /** {@inheritDoc} */ @Override public int partitions() { - return cctx.config().getAffinity().partitions(); + CacheConfiguration ccfg = cctx.config(); + + if (ccfg == null) + throw new IgniteException(FAILED_TO_FIND_CACHE_ERR_MSG + cctx.name()); + + return ccfg.getAffinity().partitions(); } /** {@inheritDoc} */ @@ -151,10 +163,21 @@ public class GridCacheAffinityImpl implements Affinity { @Override public Object affinityKey(K key) { A.notNull(key, "key"); - if (key instanceof CacheObject && !(key instanceof BinaryObject)) - key = ((CacheObject)key).value(cctx.cacheObjectContext(), false); + if (key instanceof CacheObject && !(key instanceof BinaryObject)) { + CacheObjectContext ctx = cctx.cacheObjectContext(); + + if (ctx == null) + throw new IgniteException(FAILED_TO_FIND_CACHE_ERR_MSG + cctx.name()); + + key = ((CacheObject)key).value(ctx, false); + } + + CacheConfiguration ccfg = cctx.config(); + + if (ccfg == null) + throw new IgniteException(FAILED_TO_FIND_CACHE_ERR_MSG + cctx.name()); - return cctx.config().getAffinityMapper().affinityKey(key); + return ccfg.getAffinityMapper().affinityKey(key); } /** {@inheritDoc} */ @@ -178,17 +201,18 @@ public class GridCacheAffinityImpl implements Affinity { for (K key : keys) { ClusterNode primary = cctx.affinity().primary(key, topVer); - if (primary != null) { - Collection mapped = res.get(primary); + if (primary == null) + throw new IgniteException("Failed to get primare node [topVer=" + topVer + ", key=" + key + ']'); - if (mapped == null) { - mapped = new ArrayList<>(Math.max(keys.size() / nodesCnt, 16)); + Collection mapped = res.get(primary); - res.put(primary, mapped); - } + if (mapped == null) { + mapped = new ArrayList<>(Math.max(keys.size() / nodesCnt, 16)); - mapped.add(key); + res.put(primary, mapped); } + + mapped.add(key); } return res; @@ -216,4 +240,4 @@ public class GridCacheAffinityImpl implements Affinity { private AffinityTopologyVersion topologyVersion() { return cctx.affinity().affinityTopologyVersion(); } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityMappedTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityMappedTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityMappedTest.java index 777f843..099afd1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityMappedTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityMappedTest.java @@ -109,18 +109,18 @@ public class GridAffinityMappedTest extends GridCommonAbstractTest { //Key 0 is mapped to partition 0, first node. //Key 1 is mapped to partition 1, second node. //key 2 is mapped to partition 0, first node because mapper substitutes key 2 with affinity key 0. - Map> map = g1.cluster().mapKeysToNodes(null, F.asList(0)); + Map> map = g1.affinity(null).mapKeysToNodes(F.asList(0)); assertNotNull(map); assertEquals("Invalid map size: " + map.size(), 1, map.size()); assertEquals(F.first(map.keySet()), first); - UUID id1 = g1.cluster().mapKeyToNode(null, 1).id(); + UUID id1 = g1.affinity(null).mapKeyToNode(1).id(); assertNotNull(id1); assertEquals(second.id(), id1); - UUID id2 = g1.cluster().mapKeyToNode(null, 2).id(); + UUID id2 = g1.affinity(null).mapKeyToNode(2).id(); assertNotNull(id2); assertEquals(first.id(), id2); @@ -163,4 +163,4 @@ public class GridAffinityMappedTest extends GridCommonAbstractTest { // This mapper is stateless and needs no initialization logic. } } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java new file mode 100644 index 0000000..6fb1280 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java @@ -0,0 +1,290 @@ +/* + * 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.nio.ByteBuffer; +import java.util.Collections; +import java.util.concurrent.Callable; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.CacheObjectContext; +import org.apache.ignite.internal.processors.cache.affinity.GridCacheAffinityImpl; +import org.apache.ignite.internal.util.typedef.internal.A; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.Nullable; + +/** + * Tests usage of affinity in case when cache doesn't exist. + */ +public class GridAffinityNoCacheSelfTest extends GridCommonAbstractTest { + /** */ + public static final String EXPECTED_MSG = "Failed to find cache"; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrids(2); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + super.afterTestsStopped(); + } + + /** + * @throws Exception If failed. + */ + public void testAffinityProxyNoCache() throws Exception { + checkAffinityProxyNoCache(new Object()); + } + + /** + * @throws Exception If failed. + */ + public void testAffinityProxyNoCacheCacheObject() throws Exception { + checkAffinityProxyNoCache(new TestCacheObject(new Object())); + } + + /** + * @param key Key. + */ + private void checkAffinityProxyNoCache(Object key) { + IgniteEx ignite = grid(0); + + final Affinity affinity = ignite.affinity("noCache"); + + assertFalse("Affinity proxy instance expected", affinity instanceof GridCacheAffinityImpl); + + final ClusterNode n = ignite.cluster().localNode(); + + assertAffinityMethodsException(affinity, key, n); + } + + /** + * @throws Exception If failed. + */ + public void testAffinityImplCacheDeleted() throws Exception { + checkAffinityImplCacheDeleted(new Object()); + } + + /** + * @throws Exception If failed. + */ + public void testAffinityImplCacheDeletedCacheObject() throws Exception { + checkAffinityImplCacheDeleted(new TestCacheObject(new Object())); + } + + /** + * @param key Key. + */ + private void checkAffinityImplCacheDeleted(Object key) { + IgniteEx grid = grid(0); + + final String cacheName = "cacheToBeDeleted"; + + grid(1).getOrCreateCache(cacheName); + + Affinity affinity = grid.affinity(cacheName); + + assertTrue(affinity instanceof GridCacheAffinityImpl); + + final ClusterNode n = grid.cluster().localNode(); + + grid.cache(cacheName).destroy(); + + assertAffinityMethodsException(affinity, key, n); + } + + /** + * @param affinity Affinity. + * @param key Key. + * @param n Node. + */ + private void assertAffinityMethodsException(final Affinity affinity, final Object key, + final ClusterNode n) { + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + return affinity.affinityKey(key); + } + }, IgniteException.class, EXPECTED_MSG); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + return affinity.allPartitions(n); + } + }, IgniteException.class, EXPECTED_MSG); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + return affinity.backupPartitions(n); + } + }, IgniteException.class, EXPECTED_MSG); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + return affinity.isBackup(n, key); + } + }, IgniteException.class, EXPECTED_MSG); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + return affinity.isPrimary(n, key); + } + }, IgniteException.class, EXPECTED_MSG); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + return affinity.isPrimaryOrBackup(n, key); + } + }, IgniteException.class, EXPECTED_MSG); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + return affinity.mapKeysToNodes(Collections.singleton(key)); + } + }, IgniteException.class, EXPECTED_MSG); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + return affinity.mapKeyToPrimaryAndBackups(key); + } + }, IgniteException.class, EXPECTED_MSG); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + return affinity.mapPartitionsToNodes(Collections.singleton(0)); + } + }, IgniteException.class, EXPECTED_MSG); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + return affinity.mapPartitionToNode(0); + } + }, IgniteException.class, EXPECTED_MSG); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + return affinity.mapPartitionToPrimaryAndBackups(0); + } + }, IgniteException.class, EXPECTED_MSG); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + return affinity.mapKeyToNode(key); + } + }, IgniteException.class, EXPECTED_MSG); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + return affinity.partition(key); + } + }, IgniteException.class, EXPECTED_MSG); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + return affinity.partitions(); + } + }, IgniteException.class, EXPECTED_MSG); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + return affinity.primaryPartitions(n); + } + }, IgniteException.class, EXPECTED_MSG); + } + + /** + */ + private static class TestCacheObject implements CacheObject { + /** */ + private Object val; + + /** + * @param val Value. + */ + private TestCacheObject(Object val) { + this.val = val; + } + + /** {@inheritDoc} */ + @Nullable @Override public T value(CacheObjectContext ctx, boolean cpy) { + A.notNull(ctx, "ctx"); + + return (T)val; + } + + /** {@inheritDoc} */ + @Override public byte[] valueBytes(CacheObjectContext ctx) throws IgniteCheckedException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public byte cacheObjectType() { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public boolean isPlatformType() { + return true; + } + + /** {@inheritDoc} */ + @Override public CacheObject prepareForCache(CacheObjectContext ctx) { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public void finishUnmarshal(CacheObjectContext ctx, ClassLoader ldr) throws IgniteCheckedException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public void prepareMarshal(CacheObjectContext ctx) throws IgniteCheckedException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public byte directType() { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + throw new UnsupportedOperationException(); + } + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityP2PSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityP2PSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityP2PSelfTest.java index f96b6c6..e42c4fb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityP2PSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityP2PSelfTest.java @@ -176,13 +176,13 @@ public class GridAffinityP2PSelfTest extends GridCommonAbstractTest { //Key 0 is mapped to partition 0, first node. //Key 1 is mapped to partition 1, second node. //key 2 is mapped to partition 0, first node because mapper substitutes key 2 with affinity key 0. - Map> map = g1.cluster().mapKeysToNodes(null, F.asList(0)); + Map> map = g1.affinity(null).mapKeysToNodes(F.asList(0)); assertNotNull(map); assertEquals("Invalid map size: " + map.size(), 1, map.size()); assertEquals(F.first(map.keySet()), first); - ClusterNode n1 = g1.cluster().mapKeyToNode(null, 1); + ClusterNode n1 = g1.affinity(null).mapKeyToNode(1); assertNotNull(n1); @@ -191,7 +191,7 @@ public class GridAffinityP2PSelfTest extends GridCommonAbstractTest { assertNotNull(id1); assertEquals(second.id(), id1); - ClusterNode n2 = g1.cluster().mapKeyToNode(null, 2); + ClusterNode n2 = g1.affinity(null).mapKeyToNode(2); assertNotNull(n2); @@ -206,4 +206,4 @@ public class GridAffinityP2PSelfTest extends GridCommonAbstractTest { stopGrid(3); } } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java index 4c0e2e0..0515685 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java @@ -91,18 +91,18 @@ public class GridAffinitySelfTest extends GridCommonAbstractTest { assert caches(g1).size() == 0; assert F.first(caches(g2)).getCacheMode() == PARTITIONED; - Map> map = g1.cluster().mapKeysToNodes(null, F.asList("1")); + Map> map = g1.affinity(null).mapKeysToNodes(F.asList("1")); assertNotNull(map); assertEquals("Invalid map size: " + map.size(), 1, map.size()); assertEquals(F.first(map.keySet()), g2.cluster().localNode()); - UUID id1 = g1.cluster().mapKeyToNode(null, "2").id(); + UUID id1 = g1.affinity(null).mapKeyToNode("2").id(); assertNotNull(id1); assertEquals(g2.cluster().localNode().id(), id1); - UUID id2 = g1.cluster().mapKeyToNode(null, "3").id(); + UUID id2 = g1.affinity(null).mapKeyToNode("3").id(); assertNotNull(id2); assertEquals(g2.cluster().localNode().id(), id2); @@ -120,4 +120,4 @@ public class GridAffinitySelfTest extends GridCommonAbstractTest { } }); } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java index 1e0071e..93ff515 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java @@ -325,7 +325,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract map.put("key" + i, i); // Put in primary nodes to avoid near readers which will prevent entry from being cleared. - Map> mapped = grid(0).cluster().mapKeysToNodes(null, map.keySet()); + Map> mapped = grid(0).affinity(null).mapKeysToNodes(map.keySet()); for (int i = 0; i < gridCount(); i++) { Collection keys = mapped.get(grid(i).localNode()); @@ -338,7 +338,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract map.remove("key0"); - mapped = grid(0).cluster().mapKeysToNodes(null, map.keySet()); + mapped = grid(0).affinity(null).mapKeysToNodes(map.keySet()); for (int i = 0; i < gridCount(); i++) { // Will actually delete entry from map. http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityRoutingSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityRoutingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityRoutingSelfTest.java index a89403d..34a74eb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityRoutingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityRoutingSelfTest.java @@ -276,8 +276,8 @@ public class GridCacheAffinityRoutingSelfTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override public void applyx() throws IgniteCheckedException { - assert ignite.cluster().localNode().id().equals(ignite.cluster().mapKeyToNode(null, affKey).id()); - assert ignite.cluster().localNode().id().equals(ignite.cluster().mapKeyToNode(null, key).id()); + assert ignite.cluster().localNode().id().equals(ignite.affinity(null).mapKeyToNode(affKey).id()); + assert ignite.cluster().localNode().id().equals(ignite.affinity(null).mapKeyToNode(key).id()); } } @@ -412,10 +412,10 @@ public class GridCacheAffinityRoutingSelfTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override public Object call() throws IgniteCheckedException { - assert ignite.cluster().localNode().id().equals(ignite.cluster().mapKeyToNode(null, affKey).id()); - assert ignite.cluster().localNode().id().equals(ignite.cluster().mapKeyToNode(null, key).id()); + assert ignite.cluster().localNode().id().equals(ignite.affinity(null).mapKeyToNode(affKey).id()); + assert ignite.cluster().localNode().id().equals(ignite.affinity(null).mapKeyToNode(key).id()); return null; } } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java index 45c8c2c..69445ea 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java @@ -228,7 +228,7 @@ public class GridCacheConcurrentTxMultiNodeTest extends GridCommonAbstractTest { String terminalId = String.valueOf(++tid); // Server partition cache - UUID mappedId = srvr1.cluster().mapKeyToNode(null, terminalId).id(); + UUID mappedId = srvr1.affinity(null).mapKeyToNode(terminalId).id(); if (!srvrId.equals(mappedId)) continue; @@ -840,4 +840,4 @@ public class GridCacheConcurrentTxMultiNodeTest extends GridCommonAbstractTest { return null; } } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeAbstractSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeAbstractSelfTest.java index 73454b8..7d209c6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeAbstractSelfTest.java @@ -19,8 +19,10 @@ package org.apache.ignite.internal.processors.cache; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.Callable; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; @@ -28,6 +30,7 @@ import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.apache.ignite.transactions.Transaction; @@ -167,18 +170,24 @@ public abstract class GridCacheDaemonNodeAbstractSelfTest extends GridCommonAbst // Start daemon node. daemon = true; - Ignite g2 = startGrid(4); + final Ignite g2 = startGrid(4); for (long i = 0; i < Integer.MAX_VALUE; i = (i << 1) + 1) { // Call mapKeyToNode for normal node. - assertNotNull(g1.cluster().mapKeyToNode(null, i)); + assertNotNull(g1.affinity(null).mapKeyToNode(i)); // Call mapKeyToNode for daemon node. - assertNull(g2.cluster().mapKeyToNode(null, i)); + final long i0 = i; + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + return g2.affinity(null).mapKeyToNode(i0); + } + }, IgniteException.class, "Failed to find cache"); } } finally { stopAllGrids(); } } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java index 613e98c..db6c882 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java @@ -158,7 +158,7 @@ public class GridCacheDeploymentSelfTest extends GridCommonAbstractTest { for (int i = 0; i < 1000; i++) { key = "1" + i; - if (g1.cluster().mapKeyToNode(null, key).id().equals(g2.cluster().localNode().id())) + if (g1.affinity(null).mapKeyToNode(key).id().equals(g2.cluster().localNode().id())) break; } @@ -193,7 +193,7 @@ public class GridCacheDeploymentSelfTest extends GridCommonAbstractTest { for (int i = 0; i < 1000; i++) { key = "1" + i; - if (g1.cluster().mapKeyToNode(null, key).id().equals(g2.cluster().localNode().id())) + if (g1.affinity(null).mapKeyToNode(key).id().equals(g2.cluster().localNode().id())) break; } @@ -352,7 +352,7 @@ public class GridCacheDeploymentSelfTest extends GridCommonAbstractTest { for (int i = 0; i < 1000; i++) { key = "1" + i; - if (g1.cluster().mapKeyToNode(null, key).id().equals(g2.cluster().localNode().id())) + if (g1.affinity(null).mapKeyToNode(key).id().equals(g2.cluster().localNode().id())) break; } @@ -385,7 +385,7 @@ public class GridCacheDeploymentSelfTest extends GridCommonAbstractTest { for (int i = 0; i < 1000; i++) { key = "1" + i; - if (g1.cluster().mapKeyToNode(null, key).id().equals(g2.cluster().localNode().id())) + if (g1.affinity(null).mapKeyToNode(key).id().equals(g2.cluster().localNode().id())) break; } http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryMemorySizeSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryMemorySizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryMemorySizeSelfTest.java index 0e92f50..9e3ea7b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryMemorySizeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryMemorySizeSelfTest.java @@ -217,7 +217,7 @@ public class GridCacheEntryMemorySizeSelfTest extends GridCommonAbstractTest { while (true) { key++; - if (grid(0).cluster().mapKeyToNode(null, key).equals(grid(0).localNode())) { + if (grid(0).affinity(null).mapKeyToNode(key).equals(grid(0).localNode())) { if (i > 0) jcache(0).put(key, new Value(new byte[i * 1024])); @@ -271,7 +271,7 @@ public class GridCacheEntryMemorySizeSelfTest extends GridCommonAbstractTest { while (true) { key++; - if (grid(0).cluster().mapKeyToNode(null, key).equals(grid(0).localNode())) { + if (grid(0).affinity(null).mapKeyToNode(key).equals(grid(0).localNode())) { if (i > 0) jcache(0).put(key, new Value(new byte[i * 1024])); @@ -346,4 +346,4 @@ public class GridCacheEntryMemorySizeSelfTest extends GridCommonAbstractTest { return arr != null ? Arrays.hashCode(arr) : 0; } } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedProjectionAffinitySelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedProjectionAffinitySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedProjectionAffinitySelfTest.java index be555fb..7d2b550 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedProjectionAffinitySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedProjectionAffinitySelfTest.java @@ -88,7 +88,7 @@ public class GridCachePartitionedProjectionAffinitySelfTest extends GridCommonAb Ignite g1 = grid(1); for (int i = 0; i < 100; i++) - assertEquals(g0.cluster().mapKeyToNode(null, i).id(), g1.cluster().mapKeyToNode(null, i).id()); + assertEquals(g0.affinity(null).mapKeyToNode(i).id(), g1.affinity(null).mapKeyToNode(i).id()); } /** @throws Exception If failed. */ @@ -105,8 +105,8 @@ public class GridCachePartitionedProjectionAffinitySelfTest extends GridCommonAb g1.cluster().forNodeIds(F.asList(g0.cluster().localNode().id(), g1.cluster().localNode().id())); for (int i = 0; i < 100; i++) - assertEquals(g0Pinned.ignite().cluster().mapKeyToNode(null, i).id(), - g01Pinned.ignite().cluster().mapKeyToNode(null, i).id()); + assertEquals(g0Pinned.ignite().affinity(null).mapKeyToNode(i).id(), + g01Pinned.ignite().affinity(null).mapKeyToNode(i).id()); } /** @throws Exception If failed. */ @@ -114,4 +114,4 @@ public class GridCachePartitionedProjectionAffinitySelfTest extends GridCommonAb private void waitTopologyUpdate() throws Exception { GridTestUtils.waitTopologyUpdate(null, BACKUPS, log()); } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java index 0c0d9c1..b48bbc7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java @@ -303,7 +303,7 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest { try { // Dummy call to fetch affinity function from remote node - master.cluster().mapKeyToNode(CACHE_NAME, "Dummy"); + master.affinity(CACHE_NAME).mapKeyToNode("Dummy"); Random rnd = new Random(); @@ -517,7 +517,7 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest { IgniteCompute comp = compute(master.cluster().forPredicate(workerNodesFilter)).withAsync(); for (Integer key : testKeys) { - ClusterNode mappedNode = master.cluster().mapKeyToNode(CACHE_NAME, key); + ClusterNode mappedNode = master.affinity(CACHE_NAME).mapKeyToNode(key); UUID nodeId = mappedNode.id(); http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtMultiBackupTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtMultiBackupTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtMultiBackupTest.java index 03b8af0..e1e5315 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtMultiBackupTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtMultiBackupTest.java @@ -70,7 +70,7 @@ public class GridCacheDhtMultiBackupTest extends GridCommonAbstractTest { for (int key = 0; key < 1000; key++) { SampleKey key1 = new SampleKey(key); - if (!g.cluster().localNode().id().equals(g.cluster().mapKeyToNode("partitioned", key1).id())) { + if (!g.cluster().localNode().id().equals(g.affinity("partitioned").mapKeyToNode(key1).id())) { cache.put(key1, new SampleValue(key)); cnt++; @@ -133,4 +133,4 @@ public class GridCacheDhtMultiBackupTest extends GridCommonAbstractTest { return obj instanceof SampleValue && ((SampleValue)obj).val == val; } } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java index 72c2ae8..c7e53e8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java @@ -148,7 +148,7 @@ public class GridCacheNearOnlyTopologySelfTest extends GridCommonAbstractTest { Ignite compute = startGrid(4); for (int i = 0; i < 100; i++) { - ClusterNode node = compute.cluster().mapKeyToNode(null, i); + ClusterNode node = compute.affinity(null).mapKeyToNode(i); assertFalse("For key: " + i, node.id().equals(compute.cluster().localNode().id())); assertFalse("For key: " + i, node.id().equals(grid(0).localNode().id())); @@ -250,4 +250,4 @@ public class GridCacheNearOnlyTopologySelfTest extends GridCommonAbstractTest { stopAllGrids(); } } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxMultiNodeSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxMultiNodeSelfTest.java index 1b28a4a..c246049 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxMultiNodeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxMultiNodeSelfTest.java @@ -104,7 +104,7 @@ public class GridCacheNearTxMultiNodeSelfTest extends GridCommonAbstractTest { try { Integer mainKey = 0; - ClusterNode priNode = ignite.cluster().mapKeyToNode(null, mainKey); + ClusterNode priNode = ignite.affinity(null).mapKeyToNode(mainKey); ClusterNode backupNode = F.first(F.view(ignite.affinity(null).mapKeyToPrimaryAndBackups(mainKey), F.notIn(F.asList(priNode)))); ClusterNode otherNode = F.first(ignite.cluster().forPredicate(F.notIn(F.asList(priNode, backupNode))).nodes()); @@ -250,4 +250,4 @@ public class GridCacheNearTxMultiNodeSelfTest extends GridCommonAbstractTest { for (IgniteInternalTx tx : txs) assert tx.done() : "Transaction is not finished: " + tx; } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/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 068f2ed..a7eaa33 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 @@ -92,10 +92,10 @@ public class GridCachePartitionedExplicitLockNodeFailureSelfTest extends GridCom Integer key = 0; - while (grid(idx).cluster().mapKeyToNode(null, key).id().equals(grid(0).localNode().id())) + while (grid(idx).affinity(null).mapKeyToNode(key).id().equals(grid(0).localNode().id())) key++; - ClusterNode node = grid(idx).cluster().mapKeyToNode(null, key); + ClusterNode node = grid(idx).affinity(null).mapKeyToNode(key); info("Primary node for key [id=" + node.id() + ", order=" + node.order() + ", key=" + key + ']'); @@ -158,4 +158,4 @@ public class GridCachePartitionedExplicitLockNodeFailureSelfTest extends GridCom break; } } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java index db59a7f..5abb98d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java @@ -70,10 +70,10 @@ import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; -import org.jsr166.ConcurrentLinkedDeque8; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.SECONDS; + import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.cache.CacheMode.LOCAL; @@ -581,7 +581,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo int key = 0; while (true) { - ClusterNode n = grid(0).cluster().mapKeyToNode(null, key); + ClusterNode n = grid(0).affinity(null).mapKeyToNode(key); assert n != null; @@ -655,7 +655,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo int key = 0; while (true) { - ClusterNode n = grid(0).cluster().mapKeyToNode(null, key); + ClusterNode n = grid(0).affinity(null).mapKeyToNode(key); assert n != null; @@ -741,7 +741,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo int key = 0; while (true) { - ClusterNode n = grid(0).cluster().mapKeyToNode(null, key); + ClusterNode n = grid(0).affinity(null).mapKeyToNode(key); assert n != null; @@ -1117,4 +1117,4 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo // No-op. } } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java index 2f40c8b..8383a18 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java @@ -214,7 +214,7 @@ public class IgfsStreamsSelfTest extends IgfsCommonAbstractTest { while (true) { affKey = new IgniteUuid(uuid, idx); - if (grid(0).cluster().mapKeyToNode(DATA_CACHE_NAME, affKey).id().equals(grid(0).localNode().id())) + if (grid(0).affinity(DATA_CACHE_NAME).mapKeyToNode(affKey).id().equals(grid(0).localNode().id())) break; idx++; @@ -488,4 +488,4 @@ public class IgfsStreamsSelfTest extends IgfsCommonAbstractTest { "[size=%7d, rate=%3.1f MB/sec]", expSize, expSize * 1000. / time / 1024 / 1024)); } } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/test/java/org/apache/ignite/loadtests/dsi/GridDsiClient.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/dsi/GridDsiClient.java b/modules/core/src/test/java/org/apache/ignite/loadtests/dsi/GridDsiClient.java index 2c8e6c4..a065580 100644 --- a/modules/core/src/test/java/org/apache/ignite/loadtests/dsi/GridDsiClient.java +++ b/modules/core/src/test/java/org/apache/ignite/loadtests/dsi/GridDsiClient.java @@ -272,7 +272,7 @@ public class GridDsiClient implements Callable { String terminalId = String.valueOf(++tid); // Server partition cache. - if (!srvrId.equals(g.cluster().mapKeyToNode("PARTITIONED_CACHE", terminalId).id())) + if (!srvrId.equals(g.affinity("PARTITIONED_CACHE").mapKeyToNode(terminalId).id())) continue; if (terminalsPerSrv < srvMaxNoTerminals) { @@ -428,4 +428,4 @@ public class GridDsiClient implements Callable { fileLock.close(); } } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridCacheDhtLockBackupSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridCacheDhtLockBackupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridCacheDhtLockBackupSelfTest.java index 3214a3a..f600e99 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridCacheDhtLockBackupSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridCacheDhtLockBackupSelfTest.java @@ -110,7 +110,7 @@ public class GridCacheDhtLockBackupSelfTest extends GridCommonAbstractTest { Ignite ignite2 = startGridWithSpi(2, new TestCommunicationSpi(GridNearUnlockRequest.class, 1000)); - if (!ignite1.cluster().mapKeyToNode(null, kv).id().equals(ignite1.cluster().localNode().id())) { + if (!ignite1.affinity(null).mapKeyToNode(kv).id().equals(ignite1.cluster().localNode().id())) { Ignite tmp = ignite1; ignite1 = ignite2; ignite2 = tmp; @@ -287,4 +287,4 @@ public class GridCacheDhtLockBackupSelfTest extends GridCommonAbstractTest { super.notifyListener(sndId, msg, msgC); } } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/550a4ea7/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 e2c7e26..699ad0c 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.GridAffinityNoCacheSelfTest; import org.apache.ignite.internal.GridAffinitySelfTest; import org.apache.ignite.internal.GridAlwaysFailoverSpiFailSelfTest; import org.apache.ignite.internal.GridCancelOnGridStopSelfTest; @@ -137,6 +138,7 @@ public class IgniteComputeGridTestSuite { suite.addTestSuite(GridFailoverTaskWithPredicateSelfTest.class); suite.addTestSuite(GridProjectionLocalJobMultipleArgumentsSelfTest.class); suite.addTestSuite(GridAffinitySelfTest.class); + suite.addTestSuite(GridAffinityNoCacheSelfTest.class); suite.addTestSuite(GridEventStorageRuntimeConfigurationSelfTest.class); suite.addTestSuite(GridMultinodeRedeployContinuousModeSelfTest.class); suite.addTestSuite(GridMultinodeRedeploySharedModeSelfTest.class); @@ -149,4 +151,4 @@ public class IgniteComputeGridTestSuite { return suite; } -} \ No newline at end of file +}