From fc6cdf8cff5236254c9f3d4805561ca0baa8518d Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Tue, 27 Feb 2018 18:58:38 +0300 Subject: [PATCH 01/44] ignite-5357: neineighborhoods selection logic was added --- .../dht/GridPartitionedSingleGetFuture.java | 27 ++++++++++++------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java index 4e34bcbe805be..b6b624aec13bb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java @@ -19,14 +19,18 @@ import java.util.Collection; import java.util.List; +import java.util.Optional; import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.IgniteDiagnosticAware; import org.apache.ignite.internal.IgniteDiagnosticPrepareContext; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; @@ -48,6 +52,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetResponse; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.CIX1; @@ -718,16 +723,20 @@ private ClusterTopologyServerNotFoundException serverNotFoundError(AffinityTopol * @return Affinity node to get key from. */ @Nullable private ClusterNode affinityNode(List affNodes) { - if (!canRemap) { - for (ClusterNode node : affNodes) { - if (cctx.discovery().alive(node)) - return node; - } - - return null; - } - else + if (!canRemap || !cctx.config().isReadFromBackup()) return affNodes.get(0); + + List nodes = affNodes + .parallelStream() + .filter(node -> cctx.discovery().alive(node)) + .collect(Collectors.toList()); + + Optional nodeOptional = nodes + .parallelStream() + .filter(node -> IgniteUtils.sameMacs(cctx.localNode(), node)) + .findAny(); + + return nodeOptional.orElseGet(() -> nodes.get(ThreadLocalRandom.current().nextInt(nodes.size()))); } /** {@inheritDoc} */ From 7fe3b609bd5c5adfcca26516958da32796fc0be6 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Tue, 27 Feb 2018 19:01:02 +0300 Subject: [PATCH 02/44] ignite-5357: fix statement with 'canRemap' flag --- .../cache/distributed/dht/GridPartitionedSingleGetFuture.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java index b6b624aec13bb..054571fd08088 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java @@ -723,7 +723,7 @@ private ClusterTopologyServerNotFoundException serverNotFoundError(AffinityTopol * @return Affinity node to get key from. */ @Nullable private ClusterNode affinityNode(List affNodes) { - if (!canRemap || !cctx.config().isReadFromBackup()) + if (canRemap || !cctx.config().isReadFromBackup()) return affNodes.get(0); List nodes = affNodes From 9c07ad741c1fcc2a051b77207d7f939b03a7d668 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Tue, 27 Feb 2018 19:09:29 +0300 Subject: [PATCH 03/44] ignite-5357: code style fix --- .../distributed/dht/GridPartitionedSingleGetFuture.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java index 054571fd08088..c13d9f3b860a3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java @@ -726,13 +726,11 @@ private ClusterTopologyServerNotFoundException serverNotFoundError(AffinityTopol if (canRemap || !cctx.config().isReadFromBackup()) return affNodes.get(0); - List nodes = affNodes - .parallelStream() + List nodes = affNodes.parallelStream() .filter(node -> cctx.discovery().alive(node)) .collect(Collectors.toList()); - Optional nodeOptional = nodes - .parallelStream() + Optional nodeOptional = nodes.parallelStream() .filter(node -> IgniteUtils.sameMacs(cctx.localNode(), node)) .findAny(); From 65546ae4f60d13bcbd49fa2c526ed00d63dd2b90 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Tue, 27 Feb 2018 19:30:09 +0300 Subject: [PATCH 04/44] ignite-5357: replaced parallelStream with stream --- .../cache/distributed/dht/GridPartitionedSingleGetFuture.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java index c13d9f3b860a3..a080a15ee778a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java @@ -726,11 +726,11 @@ private ClusterTopologyServerNotFoundException serverNotFoundError(AffinityTopol if (canRemap || !cctx.config().isReadFromBackup()) return affNodes.get(0); - List nodes = affNodes.parallelStream() + List nodes = affNodes.stream() .filter(node -> cctx.discovery().alive(node)) .collect(Collectors.toList()); - Optional nodeOptional = nodes.parallelStream() + Optional nodeOptional = nodes.stream() .filter(node -> IgniteUtils.sameMacs(cctx.localNode(), node)) .findAny(); From f6197a0e3994a671debb802474cde042f4a1749d Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Tue, 27 Feb 2018 20:18:22 +0300 Subject: [PATCH 05/44] ignite-5357: fix nullable logic --- .../cache/distributed/dht/GridPartitionedSingleGetFuture.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java index a080a15ee778a..d02fd8a6616ac 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java @@ -730,6 +730,9 @@ private ClusterTopologyServerNotFoundException serverNotFoundError(AffinityTopol .filter(node -> cctx.discovery().alive(node)) .collect(Collectors.toList()); + if (nodes.isEmpty()) + return null; + Optional nodeOptional = nodes.stream() .filter(node -> IgniteUtils.sameMacs(cctx.localNode(), node)) .findAny(); From b35ddb9cf8d15e3739ee55ce830cf9e442687a7e Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Tue, 27 Feb 2018 20:20:12 +0300 Subject: [PATCH 06/44] ignite-5357: remove unnecessary imports --- .../cache/distributed/dht/GridPartitionedSingleGetFuture.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java index d02fd8a6616ac..be196dcc58603 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java @@ -30,7 +30,6 @@ import org.apache.ignite.internal.IgniteDiagnosticAware; import org.apache.ignite.internal.IgniteDiagnosticPrepareContext; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; @@ -52,7 +51,6 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetResponse; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; -import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.CIX1; @@ -734,7 +732,7 @@ private ClusterTopologyServerNotFoundException serverNotFoundError(AffinityTopol return null; Optional nodeOptional = nodes.stream() - .filter(node -> IgniteUtils.sameMacs(cctx.localNode(), node)) + .filter(node -> U.sameMacs(cctx.localNode(), node)) .findAny(); return nodeOptional.orElseGet(() -> nodes.get(ThreadLocalRandom.current().nextInt(nodes.size()))); From 518a083913d7a4dbd26faf96cc0d3ce1f89dc60d Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Tue, 27 Feb 2018 20:36:16 +0300 Subject: [PATCH 07/44] ignite-5357: skip primary node condition was added --- .../cache/distributed/dht/GridPartitionedSingleGetFuture.java | 1 + 1 file changed, 1 insertion(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java index be196dcc58603..85e40bb9bbdd7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java @@ -725,6 +725,7 @@ private ClusterTopologyServerNotFoundException serverNotFoundError(AffinityTopol return affNodes.get(0); List nodes = affNodes.stream() + .skip(1) .filter(node -> cctx.discovery().alive(node)) .collect(Collectors.toList()); From ded2441863ff9582947ae20a48161b366b793ca7 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Wed, 28 Feb 2018 18:36:24 +0300 Subject: [PATCH 08/44] ignite-5357: refactoring --- .../processors/cache/GridCacheUtils.java | 29 ++++++++++++++++++ .../dht/CacheDistributedGetFutureAdapter.java | 13 ++------ .../dht/GridPartitionedSingleGetFuture.java | 30 +------------------ 3 files changed, 32 insertions(+), 40 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index 0926a70cc2ae3..d17fe9ac4ce9c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -31,6 +31,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicBoolean; import javax.cache.Cache; import javax.cache.CacheException; @@ -459,6 +460,34 @@ public static Collection affinityNodes(GridCacheContext ctx, Affini return ctx.discovery().cacheGroupAffinityNodes(ctx.groupId(), topVer); } + /** + * Affinity node to send get request to. + * + * @param affNodes All affinity nodes. + * @return Affinity node to get key from. + */ + @Nullable public static ClusterNode affinityNode(GridCacheContext ctx, List affNodes, + boolean canRemap) { + if (!ctx.config().isReadFromBackup()) + return affNodes.get(0); + + int r = ThreadLocalRandom.current().nextInt(affNodes.size()); + + ClusterNode n0 = null; + + for (ClusterNode node : affNodes) { + if (canRemap || ctx.discovery().alive(node)) { + if (U.sameMacs(ctx.localNode(), node)) + return node; + + if (r-- == 0 || n0 == null) + n0 = node; + } + } + + return n0; + } + /** * Checks if near cache is enabled for cache context. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java index 2257c9f35734a..b77e5e13d1581 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java @@ -157,17 +157,8 @@ protected CacheDistributedGetFutureAdapter( * @param affNodes All affinity nodes. * @return Affinity node to get key from. */ - protected final ClusterNode affinityNode(List affNodes) { - if (!canRemap) { - for (ClusterNode node : affNodes) { - if (cctx.discovery().alive(node)) - return node; - } - - return null; - } - else - return affNodes.get(0); + @Nullable protected final ClusterNode affinityNode(List affNodes) { + return CU.affinityNode(cctx, affNodes, canRemap); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java index 85e40bb9bbdd7..61eee89cd4cdd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java @@ -19,11 +19,8 @@ import java.util.Collection; import java.util.List; -import java.util.Optional; import java.util.UUID; -import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicReference; -import java.util.stream.Collectors; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cluster.ClusterNode; @@ -350,7 +347,7 @@ private void map(final AffinityTopologyVersion topVer) { } } - ClusterNode affNode = affinityNode(affNodes); + ClusterNode affNode = CU.affinityNode(cctx, affNodes, canRemap); if (affNode == null) { onDone(serverNotFoundError(topVer)); @@ -714,31 +711,6 @@ private ClusterTopologyServerNotFoundException serverNotFoundError(AffinityTopol "(all partition nodes left the grid) [topVer=" + topVer + ", cache=" + cctx.name() + ']'); } - /** - * Affinity node to send get request to. - * - * @param affNodes All affinity nodes. - * @return Affinity node to get key from. - */ - @Nullable private ClusterNode affinityNode(List affNodes) { - if (canRemap || !cctx.config().isReadFromBackup()) - return affNodes.get(0); - - List nodes = affNodes.stream() - .skip(1) - .filter(node -> cctx.discovery().alive(node)) - .collect(Collectors.toList()); - - if (nodes.isEmpty()) - return null; - - Optional nodeOptional = nodes.stream() - .filter(node -> U.sameMacs(cctx.localNode(), node)) - .findAny(); - - return nodeOptional.orElseGet(() -> nodes.get(ThreadLocalRandom.current().nextInt(nodes.size()))); - } - /** {@inheritDoc} */ @Override public IgniteUuid futureId() { return futId; From 883dbbff5d257c4eb8a3de65020c84c5c444ae54 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Wed, 28 Feb 2018 20:50:01 +0300 Subject: [PATCH 09/44] ignite-5357: javadoc fix --- .../ignite/internal/processors/cache/GridCacheUtils.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index d17fe9ac4ce9c..cf9bc7425ad98 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -461,9 +461,11 @@ public static Collection affinityNodes(GridCacheContext ctx, Affini } /** - * Affinity node to send get request to. + * Determines an affinity node to send get request to. * + * @param ctx Context. * @param affNodes All affinity nodes. + * @param canRemap Flag indicating that 'get' should be done on a locked topology version. * @return Affinity node to get key from. */ @Nullable public static ClusterNode affinityNode(GridCacheContext ctx, List affNodes, From e76013f40f17fc5ee07370b3d0d1ee7b73580460 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Wed, 28 Feb 2018 22:22:13 +0300 Subject: [PATCH 10/44] ignite-5357: refactoring --- .../processors/cache/GridCacheUtils.java | 18 +++++++++++------- .../ignite/internal/util/IgniteUtils.java | 13 +++++++++++++ 2 files changed, 24 insertions(+), 7 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index cf9bc7425ad98..3ba4df130ec7e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -473,21 +473,25 @@ public static Collection affinityNodes(GridCacheContext ctx, Affini if (!ctx.config().isReadFromBackup()) return affNodes.get(0); - int r = ThreadLocalRandom.current().nextInt(affNodes.size()); + String locMacs = ctx.localNode().attribute(IgniteNodeAttributes.ATTR_MACS); - ClusterNode n0 = null; + assert locMacs != null; + + int[] aliveIdxs = new int[affNodes.size()]; + int alive = 0; + + for (int i = 0; i < affNodes.size(); i++) { + ClusterNode node = affNodes.get(i); - for (ClusterNode node : affNodes) { if (canRemap || ctx.discovery().alive(node)) { - if (U.sameMacs(ctx.localNode(), node)) + if (U.sameMacs(locMacs, node)) return node; - if (r-- == 0 || n0 == null) - n0 = node; + aliveIdxs[alive++] = i; } } - return n0; + return alive == 0 ? null : affNodes.get(aliveIdxs[ThreadLocalRandom.current().nextInt(alive)]); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index cbe64cd97af74..5aa4176ec6360 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -2183,6 +2183,19 @@ public static boolean sameMacs(ClusterNode loc, ClusterNode rmt) { return locMacs != null && locMacs.equals(rmtMacs); } + /** + * @param mac MAC address. + * @param rmt Remote node. + * @return Whether given MAC address and node's MAC address are identical. + */ + public static boolean sameMacs(String mac, ClusterNode rmt) { + assert rmt != null; + + String rmtMacs = rmt.attribute(IgniteNodeAttributes.ATTR_MACS); + + return mac != null && mac.equals(rmtMacs); + } + /** * Gets a list of all local non-loopback IPs known to this JVM. * Note that this will include both IPv4 and IPv6 addresses (even if one "resolves" From eaaa0318dc0242dc901ad9b447bfdbd96e9c7698 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Wed, 28 Feb 2018 22:33:37 +0300 Subject: [PATCH 11/44] ignite-5357: back logic --- .../internal/processors/cache/GridCacheUtils.java | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index 3ba4df130ec7e..66e4ab2fd4a50 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -477,21 +477,24 @@ public static Collection affinityNodes(GridCacheContext ctx, Affini assert locMacs != null; - int[] aliveIdxs = new int[affNodes.size()]; - int alive = 0; + int r = ThreadLocalRandom.current().nextInt(affNodes.size()); - for (int i = 0; i < affNodes.size(); i++) { - ClusterNode node = affNodes.get(i); + ClusterNode n0 = null; + + for (ClusterNode node : affNodes) { + if (r > 0) + r--; if (canRemap || ctx.discovery().alive(node)) { if (U.sameMacs(locMacs, node)) return node; - aliveIdxs[alive++] = i; + if (r == 0 || n0 == null) + n0 = node; } } - return alive == 0 ? null : affNodes.get(aliveIdxs[ThreadLocalRandom.current().nextInt(alive)]); + return n0; } /** From 66097c4998e799df980591d1fa0126096a27267e Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Wed, 28 Feb 2018 22:36:24 +0300 Subject: [PATCH 12/44] ignite-5357: changed logic --- .../ignite/internal/processors/cache/GridCacheUtils.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index 66e4ab2fd4a50..886aadd06fc53 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -482,14 +482,13 @@ public static Collection affinityNodes(GridCacheContext ctx, Affini ClusterNode n0 = null; for (ClusterNode node : affNodes) { - if (r > 0) - r--; + r--; if (canRemap || ctx.discovery().alive(node)) { if (U.sameMacs(locMacs, node)) return node; - if (r == 0 || n0 == null) + if (r >= 0) n0 = node; } } From f6f0db629bc19b414ae15f7aea63cec3e08f9749 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Wed, 28 Feb 2018 22:37:45 +0300 Subject: [PATCH 13/44] ignite-5357: changed logic 2 --- .../apache/ignite/internal/processors/cache/GridCacheUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index 886aadd06fc53..f8abf1a8a93c4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -488,7 +488,7 @@ public static Collection affinityNodes(GridCacheContext ctx, Affini if (U.sameMacs(locMacs, node)) return node; - if (r >= 0) + if (r >= 0 || n0 == null) n0 = node; } } From 91c75ef77093e8747fd90d1b505d47aedacd711f Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Wed, 28 Feb 2018 22:54:36 +0300 Subject: [PATCH 14/44] ignite-5357: removed utils method --- .../internal/processors/cache/GridCacheUtils.java | 2 +- .../apache/ignite/internal/util/IgniteUtils.java | 13 ------------- 2 files changed, 1 insertion(+), 14 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index f8abf1a8a93c4..76b3bebacd162 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -485,7 +485,7 @@ public static Collection affinityNodes(GridCacheContext ctx, Affini r--; if (canRemap || ctx.discovery().alive(node)) { - if (U.sameMacs(locMacs, node)) + if (locMacs.equals(node.attribute(IgniteNodeAttributes.ATTR_MACS))) return node; if (r >= 0 || n0 == null) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index 5aa4176ec6360..cbe64cd97af74 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -2183,19 +2183,6 @@ public static boolean sameMacs(ClusterNode loc, ClusterNode rmt) { return locMacs != null && locMacs.equals(rmtMacs); } - /** - * @param mac MAC address. - * @param rmt Remote node. - * @return Whether given MAC address and node's MAC address are identical. - */ - public static boolean sameMacs(String mac, ClusterNode rmt) { - assert rmt != null; - - String rmtMacs = rmt.attribute(IgniteNodeAttributes.ATTR_MACS); - - return mac != null && mac.equals(rmtMacs); - } - /** * Gets a list of all local non-loopback IPs known to this JVM. * Note that this will include both IPv4 and IPv6 addresses (even if one "resolves" From 09ba7b2effd29fb9c6202958d4d019daf22e9800 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Fri, 2 Mar 2018 02:59:16 +0300 Subject: [PATCH 15/44] ignite-5357: requests distribution tests were added --- .../processors/cache/GridCacheUtils.java | 10 +- ...icatedAtomicCacheGetsDistributionTest.java | 221 ++++++++++++++++++ ...misticAtomicCacheGetsDistributionTest.java | 46 ++++ ...misticAtomicCacheGetsDistributionTest.java | 32 +++ .../IgniteBinaryObjectsTestSuite.java | 7 + 5 files changed, 312 insertions(+), 4 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalOptimisticAtomicCacheGetsDistributionTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalPessimisticAtomicCacheGetsDistributionTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index 76b3bebacd162..313fe1518a85d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -92,6 +92,7 @@ import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiInClosure; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgnitePredicate; @@ -380,6 +381,9 @@ public static boolean cheatCache(int id) { } }; + /** Cluster nodes' MAC addresses equality filter. */ + private static IgniteBiPredicate macsFilter = U::sameMacs; + /** * Ensure singleton. */ @@ -473,9 +477,7 @@ public static Collection affinityNodes(GridCacheContext ctx, Affini if (!ctx.config().isReadFromBackup()) return affNodes.get(0); - String locMacs = ctx.localNode().attribute(IgniteNodeAttributes.ATTR_MACS); - - assert locMacs != null; + ClusterNode locNode = ctx.localNode(); int r = ThreadLocalRandom.current().nextInt(affNodes.size()); @@ -485,7 +487,7 @@ public static Collection affinityNodes(GridCacheContext ctx, Affini r--; if (canRemap || ctx.discovery().alive(node)) { - if (locMacs.equals(node.attribute(IgniteNodeAttributes.ATTR_MACS))) + if (macsFilter.apply(locNode, node)) return node; if (r >= 0 || n0 == null) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java new file mode 100644 index 0000000000000..e76f1585a8085 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java @@ -0,0 +1,221 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.TransactionConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.lang.IgniteBiPredicate; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; + +import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; +import static org.apache.ignite.cache.CacheMode.REPLICATED; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; + +/** + * Tests of replicated cache's 'get' requests distribution. + */ +public class ReplicatedAtomicCacheGetsDistributionTest extends GridCacheAbstractSelfTest { + /** Cache name. */ + public static final String CACHE_NAME = "replicatedCache"; + + /** Value prefix. */ + public static final String VAL_PREFIX = "val"; + + /** */ + private static final int PRIMARY_KEYS_NUMBER = 100; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrid(getConfiguration("client").setClientMode(true)); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setTransactionConfiguration(transactionConfiguration()); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected CacheAtomicityMode atomicityMode() { + return ATOMIC; + } + + /** {@inheritDoc} */ + @Override protected int gridCount() { + return 4; + } + + /** + * Test 'get' operations requests distribution. + * + * @throws Exception In case of an error. + */ + public void testGetRequestsDistribution() throws Exception { + runTest(false); + } + + /** + * Test 'getAll' operations requests distribution. + * + * @throws Exception In case of an error. + */ + public void testGetAllRequestsDistribution() throws Exception { + runTest(true); + } + + /** + * @param batchMode Test mode. + * @throws Exception In case of an error. + */ + protected void runTest(boolean batchMode) throws Exception { + for (int idx = 0; idx < gridCount(); idx++) { + runTest(ignite(idx), batchMode); + + ignite(0).cache(CACHE_NAME).destroy(); + } + } + + /** + * @param dest Destination Ignite instance for requests distribution. + * @param batchMode Test mode. + * @throws Exception In case of an error. + */ + protected void runTest(final Ignite dest, final boolean batchMode) throws Exception { + final UUID destId = dest.cluster().localNode().id(); + + GridTestUtils.setFieldValue(new GridCacheUtils(), "macsFilter", new NodeIdFilter(destId)); + + IgniteCache cache = grid(0).createCache(replicatedCache()); + + List keys = primaryKeys(dest.cache(CACHE_NAME), PRIMARY_KEYS_NUMBER); + + for (Integer key : keys) + cache.put(key, VAL_PREFIX + key); + + cache = grid("client").getOrCreateCache(CACHE_NAME); + + try (Transaction tx = grid("client").transactions().txStart()) { + if (batchMode) { + Map results = cache.getAll(new HashSet<>(keys)); + + for (Map.Entry entry : results.entrySet()) + assertEquals(VAL_PREFIX + entry.getKey(), entry.getValue()); + } + else { + for (Integer key : keys) + assertEquals(VAL_PREFIX + key, cache.get(key)); + } + + tx.commit(); + } + + validateRequestsDistribution(destId); + } + + /** + * @param destId Destination node id. + */ + protected void validateRequestsDistribution(final UUID destId) { + for (int i = 0; i < gridCount(); i++) { + IgniteEx ignite = grid(i); + + long getsCount = ignite.cache(CACHE_NAME).localMetrics().getCacheGets(); + + if (destId.equals(ignite.localNode().id())) + assertEquals(PRIMARY_KEYS_NUMBER, getsCount); + else + assertEquals(0L, getsCount); + } + } + + /** + * @return Replicated cache configuration. + */ + private CacheConfiguration replicatedCache() { + return new CacheConfiguration(CACHE_NAME) + .setCacheMode(REPLICATED) + .setWriteSynchronizationMode(FULL_SYNC) + .setAtomicityMode(atomicityMode()) + .setReadFromBackup(true) + .setStatisticsEnabled(true); + } + + /** + * @return Transaction configuration. + */ + protected TransactionConfiguration transactionConfiguration() { + TransactionConfiguration txCfg = new TransactionConfiguration(); + + txCfg.setDefaultTxIsolation(transactionIsolation()); + txCfg.setDefaultTxConcurrency(transactionConcurrency()); + + return txCfg; + } + + /** + * @return Cache transaction isolation. + */ + protected TransactionIsolation transactionIsolation() { + return REPEATABLE_READ; + } + + /** + * @return Cache transaction concurrency. + */ + protected TransactionConcurrency transactionConcurrency() { + return PESSIMISTIC; + } + + /** */ + protected static class NodeIdFilter implements IgniteBiPredicate { + /** Priority node id. */ + private UUID id; + + /** + * @param id Priority node id. + */ + public NodeIdFilter(UUID id) { + this.id = id; + } + + /** {@inheritDoc} */ + @Override public boolean apply(ClusterNode n1, ClusterNode n2) { + return n2.id().equals(id); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalOptimisticAtomicCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalOptimisticAtomicCacheGetsDistributionTest.java new file mode 100644 index 0000000000000..783aa956232cf --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalOptimisticAtomicCacheGetsDistributionTest.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; + +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED; + +/** + * Tests of optimistic transactional replicated cache's 'get' requests distribution. + */ +public class ReplicatedTransactionalOptimisticAtomicCacheGetsDistributionTest extends ReplicatedAtomicCacheGetsDistributionTest { + /** {@inheritDoc} */ + @Override protected CacheAtomicityMode atomicityMode() { + return TRANSACTIONAL; + } + + /** {@inheritDoc} */ + @Override protected TransactionIsolation transactionIsolation() { + return READ_COMMITTED; + } + + /** {@inheritDoc} */ + @Override protected TransactionConcurrency transactionConcurrency() { + return OPTIMISTIC; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalPessimisticAtomicCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalPessimisticAtomicCacheGetsDistributionTest.java new file mode 100644 index 0000000000000..6b5fee323ccc9 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalPessimisticAtomicCacheGetsDistributionTest.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.transactions.TransactionConcurrency; + +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; + +/** + * Tests of pessimistic transactional replicated cache's 'get' requests distribution. + */ +public class ReplicatedTransactionalPessimisticAtomicCacheGetsDistributionTest extends ReplicatedTransactionalOptimisticAtomicCacheGetsDistributionTest { + /** {@inheritDoc} */ + @Override protected TransactionConcurrency transactionConcurrency() { + return PESSIMISTIC; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java index c0211be4adbd7..668dbe9efe47c 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java @@ -52,6 +52,9 @@ import org.apache.ignite.internal.binary.noncompact.BinaryObjectBuilderNonCompactSimpleNameLowerCaseMappersSelfTest; import org.apache.ignite.internal.binary.streams.BinaryHeapStreamByteOrderSelfTest; import org.apache.ignite.internal.binary.streams.BinaryOffheapStreamByteOrderSelfTest; +import org.apache.ignite.internal.processors.cache.ReplicatedAtomicCacheGetsDistributionTest; +import org.apache.ignite.internal.processors.cache.ReplicatedTransactionalOptimisticAtomicCacheGetsDistributionTest; +import org.apache.ignite.internal.processors.cache.ReplicatedTransactionalPessimisticAtomicCacheGetsDistributionTest; import org.apache.ignite.internal.processors.cache.binary.BinaryAtomicCacheLocalEntriesSelfTest; import org.apache.ignite.internal.processors.cache.binary.BinaryMetadataUpdatesFlowTest; import org.apache.ignite.internal.processors.cache.binary.BinaryTxCacheLocalEntriesSelfTest; @@ -148,6 +151,10 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridCacheBinaryObjectUserClassloaderSelfTest.class); + suite.addTestSuite(ReplicatedAtomicCacheGetsDistributionTest.class); + suite.addTestSuite(ReplicatedTransactionalOptimisticAtomicCacheGetsDistributionTest.class); + suite.addTestSuite(ReplicatedTransactionalPessimisticAtomicCacheGetsDistributionTest.class); + return suite; } } From 8a685e10f15a89eb48b4db731d6035c31dae9176 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Fri, 2 Mar 2018 09:02:33 +0300 Subject: [PATCH 16/44] ignite-5357: fix tests classes names --- ...TransactionalOptimisticCacheGetsDistributionTest.java} | 2 +- ...ransactionalPessimisticCacheGetsDistributionTest.java} | 2 +- .../ignite/testsuites/IgniteBinaryObjectsTestSuite.java | 8 ++++---- 3 files changed, 6 insertions(+), 6 deletions(-) rename modules/core/src/test/java/org/apache/ignite/internal/processors/cache/{ReplicatedTransactionalOptimisticAtomicCacheGetsDistributionTest.java => ReplicatedTransactionalOptimisticCacheGetsDistributionTest.java} (93%) rename modules/core/src/test/java/org/apache/ignite/internal/processors/cache/{ReplicatedTransactionalPessimisticAtomicCacheGetsDistributionTest.java => ReplicatedTransactionalPessimisticCacheGetsDistributionTest.java} (88%) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalOptimisticAtomicCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalOptimisticCacheGetsDistributionTest.java similarity index 93% rename from modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalOptimisticAtomicCacheGetsDistributionTest.java rename to modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalOptimisticCacheGetsDistributionTest.java index 783aa956232cf..3bc680972c050 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalOptimisticAtomicCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalOptimisticCacheGetsDistributionTest.java @@ -28,7 +28,7 @@ /** * Tests of optimistic transactional replicated cache's 'get' requests distribution. */ -public class ReplicatedTransactionalOptimisticAtomicCacheGetsDistributionTest extends ReplicatedAtomicCacheGetsDistributionTest { +public class ReplicatedTransactionalOptimisticCacheGetsDistributionTest extends ReplicatedAtomicCacheGetsDistributionTest { /** {@inheritDoc} */ @Override protected CacheAtomicityMode atomicityMode() { return TRANSACTIONAL; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalPessimisticAtomicCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalPessimisticCacheGetsDistributionTest.java similarity index 88% rename from modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalPessimisticAtomicCacheGetsDistributionTest.java rename to modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalPessimisticCacheGetsDistributionTest.java index 6b5fee323ccc9..4b385b28382b1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalPessimisticAtomicCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalPessimisticCacheGetsDistributionTest.java @@ -24,7 +24,7 @@ /** * Tests of pessimistic transactional replicated cache's 'get' requests distribution. */ -public class ReplicatedTransactionalPessimisticAtomicCacheGetsDistributionTest extends ReplicatedTransactionalOptimisticAtomicCacheGetsDistributionTest { +public class ReplicatedTransactionalPessimisticCacheGetsDistributionTest extends ReplicatedTransactionalOptimisticCacheGetsDistributionTest { /** {@inheritDoc} */ @Override protected TransactionConcurrency transactionConcurrency() { return PESSIMISTIC; diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java index 668dbe9efe47c..0d5905c605153 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java @@ -53,8 +53,8 @@ import org.apache.ignite.internal.binary.streams.BinaryHeapStreamByteOrderSelfTest; import org.apache.ignite.internal.binary.streams.BinaryOffheapStreamByteOrderSelfTest; import org.apache.ignite.internal.processors.cache.ReplicatedAtomicCacheGetsDistributionTest; -import org.apache.ignite.internal.processors.cache.ReplicatedTransactionalOptimisticAtomicCacheGetsDistributionTest; -import org.apache.ignite.internal.processors.cache.ReplicatedTransactionalPessimisticAtomicCacheGetsDistributionTest; +import org.apache.ignite.internal.processors.cache.ReplicatedTransactionalOptimisticCacheGetsDistributionTest; +import org.apache.ignite.internal.processors.cache.ReplicatedTransactionalPessimisticCacheGetsDistributionTest; import org.apache.ignite.internal.processors.cache.binary.BinaryAtomicCacheLocalEntriesSelfTest; import org.apache.ignite.internal.processors.cache.binary.BinaryMetadataUpdatesFlowTest; import org.apache.ignite.internal.processors.cache.binary.BinaryTxCacheLocalEntriesSelfTest; @@ -152,8 +152,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridCacheBinaryObjectUserClassloaderSelfTest.class); suite.addTestSuite(ReplicatedAtomicCacheGetsDistributionTest.class); - suite.addTestSuite(ReplicatedTransactionalOptimisticAtomicCacheGetsDistributionTest.class); - suite.addTestSuite(ReplicatedTransactionalPessimisticAtomicCacheGetsDistributionTest.class); + suite.addTestSuite(ReplicatedTransactionalOptimisticCacheGetsDistributionTest.class); + suite.addTestSuite(ReplicatedTransactionalPessimisticCacheGetsDistributionTest.class); return suite; } From 56579c8f5194744e6ce83103d33d034f95ee219b Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Fri, 2 Mar 2018 12:32:39 +0300 Subject: [PATCH 17/44] ignite-5357: fix node for getting primary keys --- .../cache/ReplicatedAtomicCacheGetsDistributionTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java index e76f1585a8085..159c1c411f790 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java @@ -122,7 +122,7 @@ protected void runTest(final Ignite dest, final boolean batchMode) throws Except IgniteCache cache = grid(0).createCache(replicatedCache()); - List keys = primaryKeys(dest.cache(CACHE_NAME), PRIMARY_KEYS_NUMBER); + List keys = primaryKeys(cache, PRIMARY_KEYS_NUMBER); for (Integer key : keys) cache.put(key, VAL_PREFIX + key); From a5fca8ab25bc66c39ec46843a0c65ce09ba9334a Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Fri, 2 Mar 2018 13:08:00 +0300 Subject: [PATCH 18/44] ignite-5357: code style fix: long line split --- ...catedTransactionalPessimisticCacheGetsDistributionTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalPessimisticCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalPessimisticCacheGetsDistributionTest.java index 4b385b28382b1..7bace3c74fddc 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalPessimisticCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalPessimisticCacheGetsDistributionTest.java @@ -24,7 +24,8 @@ /** * Tests of pessimistic transactional replicated cache's 'get' requests distribution. */ -public class ReplicatedTransactionalPessimisticCacheGetsDistributionTest extends ReplicatedTransactionalOptimisticCacheGetsDistributionTest { +public class ReplicatedTransactionalPessimisticCacheGetsDistributionTest + extends ReplicatedTransactionalOptimisticCacheGetsDistributionTest { /** {@inheritDoc} */ @Override protected TransactionConcurrency transactionConcurrency() { return PESSIMISTIC; From d85fd75315973b12b9d60f3ddc6ceba84147f869 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Fri, 2 Mar 2018 13:45:42 +0300 Subject: [PATCH 19/44] ignite-5357: javadoc comment was extended --- .../apache/ignite/internal/processors/cache/GridCacheUtils.java | 2 +- .../cache/distributed/dht/CacheDistributedGetFutureAdapter.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index 313fe1518a85d..29e8de69bf191 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -470,7 +470,7 @@ public static Collection affinityNodes(GridCacheContext ctx, Affini * @param ctx Context. * @param affNodes All affinity nodes. * @param canRemap Flag indicating that 'get' should be done on a locked topology version. - * @return Affinity node to get key from. + * @return Affinity node to get key from or {@code null} or {@code null} if there is no suitable alive node. */ @Nullable public static ClusterNode affinityNode(GridCacheContext ctx, List affNodes, boolean canRemap) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java index b77e5e13d1581..686324b6000f6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java @@ -155,7 +155,7 @@ protected CacheDistributedGetFutureAdapter( * Affinity node to send get request to. * * @param affNodes All affinity nodes. - * @return Affinity node to get key from. + * @return @return Affinity node to get key from or {@code null} or {@code null} if there is no suitable alive node. */ @Nullable protected final ClusterNode affinityNode(List affNodes) { return CU.affinityNode(cctx, affNodes, canRemap); From 2de55a8954cd10b55613a4229f390c5d03d2984e Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Fri, 2 Mar 2018 17:39:20 +0300 Subject: [PATCH 20/44] ignite-5357: generator tests were added --- .../processors/cache/GridCacheUtils.java | 11 +- .../dht/CacheDistributedGetFutureAdapter.java | 2 +- ...icatedAtomicCacheGetsDistributionTest.java | 105 ++++++++++++++++-- 3 files changed, 102 insertions(+), 16 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index 29e8de69bf191..a2744508264e7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -384,6 +384,9 @@ public static boolean cheatCache(int id) { /** Cluster nodes' MAC addresses equality filter. */ private static IgniteBiPredicate macsFilter = U::sameMacs; + /** ThreadLocalRandom generator closure. */ + private static IgniteClosure rndClo = bound -> ThreadLocalRandom.current().nextInt(bound); + /** * Ensure singleton. */ @@ -470,7 +473,7 @@ public static Collection affinityNodes(GridCacheContext ctx, Affini * @param ctx Context. * @param affNodes All affinity nodes. * @param canRemap Flag indicating that 'get' should be done on a locked topology version. - * @return Affinity node to get key from or {@code null} or {@code null} if there is no suitable alive node. + * @return Affinity node to get key from or {@code null} if there is no suitable alive node. */ @Nullable public static ClusterNode affinityNode(GridCacheContext ctx, List affNodes, boolean canRemap) { @@ -479,13 +482,11 @@ public static Collection affinityNodes(GridCacheContext ctx, Affini ClusterNode locNode = ctx.localNode(); - int r = ThreadLocalRandom.current().nextInt(affNodes.size()); + int r = rndClo.apply(affNodes.size()); ClusterNode n0 = null; for (ClusterNode node : affNodes) { - r--; - if (canRemap || ctx.discovery().alive(node)) { if (macsFilter.apply(locNode, node)) return node; @@ -493,6 +494,8 @@ public static Collection affinityNodes(GridCacheContext ctx, Affini if (r >= 0 || n0 == null) n0 = node; } + + r--; } return n0; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java index 686324b6000f6..1ff124931d649 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java @@ -155,7 +155,7 @@ protected CacheDistributedGetFutureAdapter( * Affinity node to send get request to. * * @param affNodes All affinity nodes. - * @return @return Affinity node to get key from or {@code null} or {@code null} if there is no suitable alive node. + * @return Affinity node to get key from or {@code null} if there is no suitable alive node. */ @Nullable protected final ClusterNode affinityNode(List affNodes) { return CU.affinityNode(cctx, affNodes, canRemap); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java index 159c1c411f790..bce3dc516857c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java @@ -18,8 +18,9 @@ package org.apache.ignite.internal.processors.cache; import java.util.HashSet; -import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.TreeSet; import java.util.UUID; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; @@ -30,6 +31,7 @@ import org.apache.ignite.configuration.TransactionConfiguration; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.lang.IgniteBiPredicate; +import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.transactions.Transaction; import org.apache.ignite.transactions.TransactionConcurrency; @@ -52,15 +54,27 @@ public class ReplicatedAtomicCacheGetsDistributionTest extends GridCacheAbstract public static final String VAL_PREFIX = "val"; /** */ - private static final int PRIMARY_KEYS_NUMBER = 100; + private static final int PRIMARY_KEYS_NUMBER = 1_000; /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { super.beforeTestsStarted(); + assert PRIMARY_KEYS_NUMBER % gridCount() == 0; + startGrid(getConfiguration("client").setClientMode(true)); } + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + IgniteCache cache = ignite(0).cache(CACHE_NAME); + + if (cache != null) + cache.destroy(); + } + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); @@ -80,13 +94,62 @@ public class ReplicatedAtomicCacheGetsDistributionTest extends GridCacheAbstract return 4; } + /** + * Test 'get' operations requests generator distribution. + * + * @throws Exception In case of an error. + */ + public void testGetRequestsGeneratorDistribution() throws Exception { + runTestGetRequestsGeneratorDistribution(false); + } + + /** + * Test 'getAll' operations requests generator distribution. + * + * @throws Exception In case of an error. + */ + public void testGetAllRequestsGeneratorDistribution() throws Exception { + runTestGetRequestsGeneratorDistribution(true); + } + + /** + * @param batchMode Test mode. + * @throws Exception In case of an error. + */ + protected void runTestGetRequestsGeneratorDistribution(boolean batchMode) throws Exception { + GridTestUtils.setFieldValue(new GridCacheUtils(), "macsFilter", new NodeIdFilter(null)); // Always false. + + GridTestUtils.setFieldValue(new GridCacheUtils(), "rndClo", new RoundRobinDistributionClosure()); + + IgniteCache cache = grid(0).createCache(replicatedCache()); + + Set keys = new TreeSet<>(primaryKeys(cache, PRIMARY_KEYS_NUMBER)); + + for (Integer key : keys) + cache.put(key, VAL_PREFIX + key); + + cache = grid("client").getOrCreateCache(CACHE_NAME); + + getAndValidateData(cache, new HashSet<>(keys), batchMode); + + int expected = PRIMARY_KEYS_NUMBER / gridCount(); + + for (int i = 0; i < gridCount(); i++) { + IgniteEx ignite = grid(i); + + long getsCount = ignite.cache(CACHE_NAME).localMetrics().getCacheGets(); + + assertEquals(expected, getsCount); + } + } + /** * Test 'get' operations requests distribution. * * @throws Exception In case of an error. */ public void testGetRequestsDistribution() throws Exception { - runTest(false); + runTestGetAllRequestsDistribution(false); } /** @@ -95,16 +158,16 @@ public void testGetRequestsDistribution() throws Exception { * @throws Exception In case of an error. */ public void testGetAllRequestsDistribution() throws Exception { - runTest(true); + runTestGetAllRequestsDistribution(true); } /** * @param batchMode Test mode. * @throws Exception In case of an error. */ - protected void runTest(boolean batchMode) throws Exception { + protected void runTestGetAllRequestsDistribution(boolean batchMode) throws Exception { for (int idx = 0; idx < gridCount(); idx++) { - runTest(ignite(idx), batchMode); + runTestGetAllRequestsDistribution(ignite(idx), batchMode); ignite(0).cache(CACHE_NAME).destroy(); } @@ -115,23 +178,34 @@ protected void runTest(boolean batchMode) throws Exception { * @param batchMode Test mode. * @throws Exception In case of an error. */ - protected void runTest(final Ignite dest, final boolean batchMode) throws Exception { + protected void runTestGetAllRequestsDistribution(final Ignite dest, final boolean batchMode) throws Exception { final UUID destId = dest.cluster().localNode().id(); GridTestUtils.setFieldValue(new GridCacheUtils(), "macsFilter", new NodeIdFilter(destId)); IgniteCache cache = grid(0).createCache(replicatedCache()); - List keys = primaryKeys(cache, PRIMARY_KEYS_NUMBER); + Set keys = new TreeSet<>(primaryKeys(cache, PRIMARY_KEYS_NUMBER)); for (Integer key : keys) cache.put(key, VAL_PREFIX + key); cache = grid("client").getOrCreateCache(CACHE_NAME); + getAndValidateData(cache, new HashSet<>(keys), batchMode); + + validateRequestsDistribution(destId); + } + + /** + * @param cache Ignite cache. + * @param keys Keys to get. + * @param batchMode Test mode. + */ + protected void getAndValidateData(IgniteCache cache, Set keys, boolean batchMode) { try (Transaction tx = grid("client").transactions().txStart()) { if (batchMode) { - Map results = cache.getAll(new HashSet<>(keys)); + Map results = cache.getAll(keys); for (Map.Entry entry : results.entrySet()) assertEquals(VAL_PREFIX + entry.getKey(), entry.getValue()); @@ -143,8 +217,6 @@ protected void runTest(final Ignite dest, final boolean batchMode) throws Except tx.commit(); } - - validateRequestsDistribution(destId); } /** @@ -201,6 +273,17 @@ protected TransactionConcurrency transactionConcurrency() { return PESSIMISTIC; } + /** */ + private static class RoundRobinDistributionClosure implements IgniteClosure { + /** Counter. */ + private int counter; + + /** {@inheritDoc} */ + @Override public Integer apply(Integer num) { + return counter++ % num; + } + } + /** */ protected static class NodeIdFilter implements IgniteBiPredicate { /** Priority node id. */ From c0c95e0715f80c3148b77927a93ed7b75518d971 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Fri, 2 Mar 2018 18:21:34 +0300 Subject: [PATCH 21/44] ignite-5357: lamda expressions were unwraped --- .../internal/processors/cache/GridCacheUtils.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index a2744508264e7..e2e46e7be6fbd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -382,10 +382,18 @@ public static boolean cheatCache(int id) { }; /** Cluster nodes' MAC addresses equality filter. */ - private static IgniteBiPredicate macsFilter = U::sameMacs; + private static IgniteBiPredicate macsFilter = new IgniteBiPredicate() { + @Override public boolean apply(ClusterNode n1, ClusterNode n2) { + return U.sameMacs(n1, n2); + } + }; /** ThreadLocalRandom generator closure. */ - private static IgniteClosure rndClo = bound -> ThreadLocalRandom.current().nextInt(bound); + private static IgniteClosure rndClo = new IgniteClosure() { + @Override public Integer apply(Integer bound) { + return ThreadLocalRandom.current().nextInt(bound); + } + }; /** * Ensure singleton. From d12e14bd143e56674f9fc5d8e4f44ebbeb728dcf Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Fri, 2 Mar 2018 23:23:58 +0300 Subject: [PATCH 22/44] ignite-5357: minor fix --- .../cache/ReplicatedAtomicCacheGetsDistributionTest.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java index bce3dc516857c..f9bdd6fb97da4 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java @@ -17,7 +17,6 @@ package org.apache.ignite.internal.processors.cache; -import java.util.HashSet; import java.util.Map; import java.util.Set; import java.util.TreeSet; @@ -130,7 +129,7 @@ protected void runTestGetRequestsGeneratorDistribution(boolean batchMode) throws cache = grid("client").getOrCreateCache(CACHE_NAME); - getAndValidateData(cache, new HashSet<>(keys), batchMode); + getAndValidateData(cache, keys, batchMode); int expected = PRIMARY_KEYS_NUMBER / gridCount(); @@ -192,7 +191,7 @@ protected void runTestGetAllRequestsDistribution(final Ignite dest, final boolea cache = grid("client").getOrCreateCache(CACHE_NAME); - getAndValidateData(cache, new HashSet<>(keys), batchMode); + getAndValidateData(cache, keys, batchMode); validateRequestsDistribution(destId); } From 62093d240568f18618517d9cdfb6767c1d5543a2 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Mon, 12 Mar 2018 00:08:32 +0300 Subject: [PATCH 23/44] ignite-5357: review notes fixes; refactoring --- .../processors/cache/GridCacheUtils.java | 25 ++-- ...icatedAtomicCacheGetsDistributionTest.java | 133 +++++++++--------- 2 files changed, 78 insertions(+), 80 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index e2e46e7be6fbd..7659b75e32d95 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -92,7 +92,6 @@ import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiInClosure; -import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgnitePredicate; @@ -118,6 +117,7 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC; import static org.apache.ignite.configuration.CacheConfiguration.DFLT_CACHE_MODE; import static org.apache.ignite.internal.GridTopic.TOPIC_REPLICATION; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MACS; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.READ; /** @@ -381,19 +381,8 @@ public static boolean cheatCache(int id) { } }; - /** Cluster nodes' MAC addresses equality filter. */ - private static IgniteBiPredicate macsFilter = new IgniteBiPredicate() { - @Override public boolean apply(ClusterNode n1, ClusterNode n2) { - return U.sameMacs(n1, n2); - } - }; - - /** ThreadLocalRandom generator closure. */ - private static IgniteClosure rndClo = new IgniteClosure() { - @Override public Integer apply(Integer bound) { - return ThreadLocalRandom.current().nextInt(bound); - } - }; + /** Random generator. */ + private static final ThreadLocalRandom RND = ThreadLocalRandom.current(); /** * Ensure singleton. @@ -488,15 +477,17 @@ public static Collection affinityNodes(GridCacheContext ctx, Affini if (!ctx.config().isReadFromBackup()) return affNodes.get(0); - ClusterNode locNode = ctx.localNode(); + String locMacs = ctx.localNode().attribute(ATTR_MACS); + + assert locMacs != null; - int r = rndClo.apply(affNodes.size()); + int r = RND.nextInt(affNodes.size()); ClusterNode n0 = null; for (ClusterNode node : affNodes) { if (canRemap || ctx.discovery().alive(node)) { - if (macsFilter.apply(locNode, node)) + if (locMacs.equals(node.attribute(ATTR_MACS))) return node; if (r >= 0 || n0 == null) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java index f9bdd6fb97da4..d76dba48fcfda 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java @@ -17,6 +17,9 @@ package org.apache.ignite.internal.processors.cache; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.TreeSet; @@ -29,9 +32,8 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.TransactionConfiguration; import org.apache.ignite.internal.IgniteEx; -import org.apache.ignite.lang.IgniteBiPredicate; -import org.apache.ignite.lang.IgniteClosure; -import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; import org.apache.ignite.transactions.Transaction; import org.apache.ignite.transactions.TransactionConcurrency; import org.apache.ignite.transactions.TransactionIsolation; @@ -39,6 +41,7 @@ import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; import static org.apache.ignite.cache.CacheMode.REPLICATED; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MACS; import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; @@ -47,10 +50,16 @@ */ public class ReplicatedAtomicCacheGetsDistributionTest extends GridCacheAbstractSelfTest { /** Cache name. */ - public static final String CACHE_NAME = "replicatedCache"; + private static final String CACHE_NAME = "replicatedCache"; + + /** Client nodes instance's name. */ + private static final String CLIENT_NAME = "client"; /** Value prefix. */ - public static final String VAL_PREFIX = "val"; + private static final String VAL_PREFIX = "val"; + + /** Mapping MAC addresses to nodes ids. */ + private static Map macs; /** */ private static final int PRIMARY_KEYS_NUMBER = 1_000; @@ -59,9 +68,18 @@ public class ReplicatedAtomicCacheGetsDistributionTest extends GridCacheAbstract @Override protected void beforeTestsStarted() throws Exception { super.beforeTestsStarted(); + startGrid(getConfiguration(CLIENT_NAME).setClientMode(true)); + assert PRIMARY_KEYS_NUMBER % gridCount() == 0; - startGrid(getConfiguration("client").setClientMode(true)); + Map newMacs = new HashMap<>(); + + for (int idx = 0; idx < gridCount(); idx++) + newMacs.put(grid(idx).localNode().id(), "x2-xx-xx-xx-xx-x" + idx); + + newMacs.put(grid(CLIENT_NAME).localNode().id(), "x2-xx-xx-xx-xx-x" + (gridCount() + 1)); + + macs = Collections.unmodifiableMap(newMacs); } /** {@inheritDoc} */ @@ -72,6 +90,28 @@ public class ReplicatedAtomicCacheGetsDistributionTest extends GridCacheAbstract if (cache != null) cache.destroy(); + + // Setting different MAC addresses for all nodes + setMacAddress(G.allGrids(), macs); + } + + /** + * @param instances Started Ignite instances. + * @param macs Mapping MAC addresses to UUID. + */ + private void setMacAddress(List instances, Map macs) { + for (Ignite ignite : instances) { + for (ClusterNode node : ignite.cluster().nodes()) { + String mac = macs.get(node.id()); + + assert mac != null; + + Map attrs = new HashMap<>(node.attributes()); + attrs.put(ATTR_MACS, mac); + + ((TcpDiscoveryNode)node).setAttributes(attrs); + } + } } /** {@inheritDoc} */ @@ -97,6 +137,7 @@ public class ReplicatedAtomicCacheGetsDistributionTest extends GridCacheAbstract * Test 'get' operations requests generator distribution. * * @throws Exception In case of an error. + * @see #runTestGetRequestsGeneratorDistribution(boolean) */ public void testGetRequestsGeneratorDistribution() throws Exception { runTestGetRequestsGeneratorDistribution(false); @@ -106,6 +147,7 @@ public void testGetRequestsGeneratorDistribution() throws Exception { * Test 'getAll' operations requests generator distribution. * * @throws Exception In case of an error. + * @see #runTestGetRequestsGeneratorDistribution(boolean) */ public void testGetAllRequestsGeneratorDistribution() throws Exception { runTestGetRequestsGeneratorDistribution(true); @@ -116,10 +158,6 @@ public void testGetAllRequestsGeneratorDistribution() throws Exception { * @throws Exception In case of an error. */ protected void runTestGetRequestsGeneratorDistribution(boolean batchMode) throws Exception { - GridTestUtils.setFieldValue(new GridCacheUtils(), "macsFilter", new NodeIdFilter(null)); // Always false. - - GridTestUtils.setFieldValue(new GridCacheUtils(), "rndClo", new RoundRobinDistributionClosure()); - IgniteCache cache = grid(0).createCache(replicatedCache()); Set keys = new TreeSet<>(primaryKeys(cache, PRIMARY_KEYS_NUMBER)); @@ -127,60 +165,58 @@ protected void runTestGetRequestsGeneratorDistribution(boolean batchMode) throws for (Integer key : keys) cache.put(key, VAL_PREFIX + key); - cache = grid("client").getOrCreateCache(CACHE_NAME); + cache = grid(CLIENT_NAME).getOrCreateCache(CACHE_NAME); getAndValidateData(cache, keys, batchMode); - int expected = PRIMARY_KEYS_NUMBER / gridCount(); - for (int i = 0; i < gridCount(); i++) { IgniteEx ignite = grid(i); long getsCount = ignite.cache(CACHE_NAME).localMetrics().getCacheGets(); - assertEquals(expected, getsCount); + assertTrue(getsCount > 0); } } /** - * Test 'get' operations requests distribution. + * Tests that the 'get' operation requests are routed to node with same MAC address as at requester. * * @throws Exception In case of an error. + * @see #runTestGetAllRequestsDistribution(UUID, boolean) */ public void testGetRequestsDistribution() throws Exception { - runTestGetAllRequestsDistribution(false); + UUID destId = grid(0).localNode().id(); + + runTestGetAllRequestsDistribution(destId, false); } /** - * Test 'getAll' operations requests distribution. + * Tests that the 'getAll' operation requests are routed to node with same MAC address as at requester. * * @throws Exception In case of an error. + * @see #runTestGetAllRequestsDistribution(UUID, boolean) */ public void testGetAllRequestsDistribution() throws Exception { - runTestGetAllRequestsDistribution(true); + UUID destId = grid(gridCount() - 1).localNode().id(); + + runTestGetAllRequestsDistribution(destId, true); } /** + * Tests that the 'get' and 'getAll requests are routed to node with same MAC address as at requester. + * + * @param destId Destination Ignite instance id for requests distribution. * @param batchMode Test mode. * @throws Exception In case of an error. */ - protected void runTestGetAllRequestsDistribution(boolean batchMode) throws Exception { - for (int idx = 0; idx < gridCount(); idx++) { - runTestGetAllRequestsDistribution(ignite(idx), batchMode); + protected void runTestGetAllRequestsDistribution(final UUID destId, final boolean batchMode) throws Exception { + String clientMac = macs.get(grid(CLIENT_NAME).localNode().id()); - ignite(0).cache(CACHE_NAME).destroy(); - } - } + Map newMacs = new HashMap<>(macs); - /** - * @param dest Destination Ignite instance for requests distribution. - * @param batchMode Test mode. - * @throws Exception In case of an error. - */ - protected void runTestGetAllRequestsDistribution(final Ignite dest, final boolean batchMode) throws Exception { - final UUID destId = dest.cluster().localNode().id(); + assert newMacs.put(destId, clientMac) != null; - GridTestUtils.setFieldValue(new GridCacheUtils(), "macsFilter", new NodeIdFilter(destId)); + setMacAddress(G.allGrids(), newMacs); IgniteCache cache = grid(0).createCache(replicatedCache()); @@ -189,7 +225,7 @@ protected void runTestGetAllRequestsDistribution(final Ignite dest, final boolea for (Integer key : keys) cache.put(key, VAL_PREFIX + key); - cache = grid("client").getOrCreateCache(CACHE_NAME); + cache = grid(CLIENT_NAME).getOrCreateCache(CACHE_NAME); getAndValidateData(cache, keys, batchMode); @@ -202,7 +238,7 @@ protected void runTestGetAllRequestsDistribution(final Ignite dest, final boolea * @param batchMode Test mode. */ protected void getAndValidateData(IgniteCache cache, Set keys, boolean batchMode) { - try (Transaction tx = grid("client").transactions().txStart()) { + try (Transaction tx = grid(CLIENT_NAME).transactions().txStart()) { if (batchMode) { Map results = cache.getAll(keys); @@ -271,33 +307,4 @@ protected TransactionIsolation transactionIsolation() { protected TransactionConcurrency transactionConcurrency() { return PESSIMISTIC; } - - /** */ - private static class RoundRobinDistributionClosure implements IgniteClosure { - /** Counter. */ - private int counter; - - /** {@inheritDoc} */ - @Override public Integer apply(Integer num) { - return counter++ % num; - } - } - - /** */ - protected static class NodeIdFilter implements IgniteBiPredicate { - /** Priority node id. */ - private UUID id; - - /** - * @param id Priority node id. - */ - public NodeIdFilter(UUID id) { - this.id = id; - } - - /** {@inheritDoc} */ - @Override public boolean apply(ClusterNode n1, ClusterNode n2) { - return n2.id().equals(id); - } - } } From 43f485ea505e45112cf6f95321ae89c0b9c1c1ad Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Mon, 12 Mar 2018 11:28:43 +0300 Subject: [PATCH 24/44] ignite-5357: moved random generator to method --- .../ignite/internal/processors/cache/GridCacheUtils.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index 7659b75e32d95..8e779123dc3bb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -381,9 +381,6 @@ public static boolean cheatCache(int id) { } }; - /** Random generator. */ - private static final ThreadLocalRandom RND = ThreadLocalRandom.current(); - /** * Ensure singleton. */ @@ -481,7 +478,7 @@ public static Collection affinityNodes(GridCacheContext ctx, Affini assert locMacs != null; - int r = RND.nextInt(affNodes.size()); + int r = ThreadLocalRandom.current().nextInt(affNodes.size()); ClusterNode n0 = null; From c0fc039eba4b5596d496bb69c84fe2df10df680c Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Mon, 12 Mar 2018 11:55:09 +0300 Subject: [PATCH 25/44] ignite-5357: review notes fixes --- .../dht/CacheDistributedGetFutureAdapter.java | 10 ---------- .../distributed/dht/GridPartitionedGetFuture.java | 2 +- .../cache/distributed/near/GridNearGetFuture.java | 2 +- 3 files changed, 2 insertions(+), 12 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java index 1ff124931d649..d9c4b3b8028a0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java @@ -151,16 +151,6 @@ protected CacheDistributedGetFutureAdapter( futId = IgniteUuid.randomUuid(); } - /** - * Affinity node to send get request to. - * - * @param affNodes All affinity nodes. - * @return Affinity node to get key from or {@code null} if there is no suitable alive node. - */ - @Nullable protected final ClusterNode affinityNode(List affNodes) { - return CU.affinityNode(cctx, affNodes, canRemap); - } - /** * @param part Partition. * @return {@code True} if partition is in owned state. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java index 179a8ae9c5da1..1be56fccac97d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java @@ -407,7 +407,7 @@ private boolean map( } } - ClusterNode node = affinityNode(affNodes); + ClusterNode node = CU.affinityNode(cctx, affNodes, canRemap); if (node == null) { onDone(serverNotFoundError(topVer)); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java index 29dd12fd4be84..1552132f0c6e9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java @@ -490,7 +490,7 @@ private Map map( } } - ClusterNode affNode = affinityNode(affNodes); + ClusterNode affNode = CU.affinityNode(cctx, affNodes, canRemap); if (affNode == null) { onDone(serverNotFoundError(topVer)); From 5a98fb8bcef6c6134221d0d7e418e0f325b0e02f Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Mon, 12 Mar 2018 14:06:03 +0300 Subject: [PATCH 26/44] ignite-5357: review notes fixes: remove static map --- ...icatedAtomicCacheGetsDistributionTest.java | 105 ++++++++++-------- 1 file changed, 59 insertions(+), 46 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java index d76dba48fcfda..a4d2cca4ae378 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java @@ -17,7 +17,6 @@ package org.apache.ignite.internal.processors.cache; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -58,9 +57,6 @@ public class ReplicatedAtomicCacheGetsDistributionTest extends GridCacheAbstract /** Value prefix. */ private static final String VAL_PREFIX = "val"; - /** Mapping MAC addresses to nodes ids. */ - private static Map macs; - /** */ private static final int PRIMARY_KEYS_NUMBER = 1_000; @@ -71,15 +67,6 @@ public class ReplicatedAtomicCacheGetsDistributionTest extends GridCacheAbstract startGrid(getConfiguration(CLIENT_NAME).setClientMode(true)); assert PRIMARY_KEYS_NUMBER % gridCount() == 0; - - Map newMacs = new HashMap<>(); - - for (int idx = 0; idx < gridCount(); idx++) - newMacs.put(grid(idx).localNode().id(), "x2-xx-xx-xx-xx-x" + idx); - - newMacs.put(grid(CLIENT_NAME).localNode().id(), "x2-xx-xx-xx-xx-x" + (gridCount() + 1)); - - macs = Collections.unmodifiableMap(newMacs); } /** {@inheritDoc} */ @@ -92,26 +79,14 @@ public class ReplicatedAtomicCacheGetsDistributionTest extends GridCacheAbstract cache.destroy(); // Setting different MAC addresses for all nodes - setMacAddress(G.allGrids(), macs); - } - - /** - * @param instances Started Ignite instances. - * @param macs Mapping MAC addresses to UUID. - */ - private void setMacAddress(List instances, Map macs) { - for (Ignite ignite : instances) { - for (ClusterNode node : ignite.cluster().nodes()) { - String mac = macs.get(node.id()); + Map macs = getClusterMacs(); - assert mac != null; + int idx = 0; - Map attrs = new HashMap<>(node.attributes()); - attrs.put(ATTR_MACS, mac); + for (Map.Entry entry : macs.entrySet()) + entry.setValue("x2-xx-xx-xx-xx-x" + idx++); - ((TcpDiscoveryNode)node).setAttributes(attrs); - } - } + setMacAddress(G.allGrids(), macs); } /** {@inheritDoc} */ @@ -210,13 +185,13 @@ public void testGetAllRequestsDistribution() throws Exception { * @throws Exception In case of an error. */ protected void runTestGetAllRequestsDistribution(final UUID destId, final boolean batchMode) throws Exception { - String clientMac = macs.get(grid(CLIENT_NAME).localNode().id()); + Map macs = getClusterMacs(); - Map newMacs = new HashMap<>(macs); + String clientMac = macs.get(grid(CLIENT_NAME).localNode().id()); - assert newMacs.put(destId, clientMac) != null; + assert macs.put(destId, clientMac) != null; - setMacAddress(G.allGrids(), newMacs); + setMacAddress(G.allGrids(), macs); IgniteCache cache = grid(0).createCache(replicatedCache()); @@ -270,18 +245,6 @@ protected void validateRequestsDistribution(final UUID destId) { } } - /** - * @return Replicated cache configuration. - */ - private CacheConfiguration replicatedCache() { - return new CacheConfiguration(CACHE_NAME) - .setCacheMode(REPLICATED) - .setWriteSynchronizationMode(FULL_SYNC) - .setAtomicityMode(atomicityMode()) - .setReadFromBackup(true) - .setStatisticsEnabled(true); - } - /** * @return Transaction configuration. */ @@ -307,4 +270,54 @@ protected TransactionIsolation transactionIsolation() { protected TransactionConcurrency transactionConcurrency() { return PESSIMISTIC; } + + /** + * @return Replicated cache configuration. + */ + private CacheConfiguration replicatedCache() { + return new CacheConfiguration(CACHE_NAME) + .setCacheMode(REPLICATED) + .setWriteSynchronizationMode(FULL_SYNC) + .setAtomicityMode(atomicityMode()) + .setReadFromBackup(true) + .setStatisticsEnabled(true); + } + + /** + * @param instances Started Ignite instances. + * @param macs Mapping MAC addresses to UUID. + */ + private void setMacAddress(List instances, Map macs) { + for (Ignite ignite : instances) { + for (ClusterNode node : ignite.cluster().nodes()) { + String mac = macs.get(node.id()); + + assert mac != null; + + Map attrs = new HashMap<>(node.attributes()); + attrs.put(ATTR_MACS, mac); + + ((TcpDiscoveryNode)node).setAttributes(attrs); + } + } + } + + /** + * @return Cluster nodes MAC addresses. + */ + private Map getClusterMacs() { + Map macs = new HashMap<>(); + + for (Ignite ignite : G.allGrids()) { + ClusterNode node = ignite.cluster().localNode(); + + String mac = node.attribute(ATTR_MACS); + + assert mac != null; + + macs.put(node.id(), mac); + } + + return macs; + } } From 19435e56ccd00807999ab67f73e7c2603cb87d9a Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Mon, 12 Mar 2018 16:44:02 +0300 Subject: [PATCH 27/44] ignite-5357: removed deprecated assertion --- .../cache/ReplicatedAtomicCacheGetsDistributionTest.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java index a4d2cca4ae378..9d38447fddf63 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java @@ -65,8 +65,6 @@ public class ReplicatedAtomicCacheGetsDistributionTest extends GridCacheAbstract super.beforeTestsStarted(); startGrid(getConfiguration(CLIENT_NAME).setClientMode(true)); - - assert PRIMARY_KEYS_NUMBER % gridCount() == 0; } /** {@inheritDoc} */ From ae1007a3a44cd9f3f86b013596d4a42879b8e988 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Mon, 12 Mar 2018 17:17:41 +0300 Subject: [PATCH 28/44] ignite-5357: review fixes --- ...ReplicatedAtomicCacheGetsDistributionTest.java | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java index 9d38447fddf63..3f1b9305e2569 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java @@ -20,7 +20,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.TreeSet; import java.util.UUID; import org.apache.ignite.Ignite; @@ -84,7 +83,7 @@ public class ReplicatedAtomicCacheGetsDistributionTest extends GridCacheAbstract for (Map.Entry entry : macs.entrySet()) entry.setValue("x2-xx-xx-xx-xx-x" + idx++); - setMacAddress(G.allGrids(), macs); + replaceMacAddresses(G.allGrids(), macs); } /** {@inheritDoc} */ @@ -133,7 +132,7 @@ public void testGetAllRequestsGeneratorDistribution() throws Exception { protected void runTestGetRequestsGeneratorDistribution(boolean batchMode) throws Exception { IgniteCache cache = grid(0).createCache(replicatedCache()); - Set keys = new TreeSet<>(primaryKeys(cache, PRIMARY_KEYS_NUMBER)); + List keys = primaryKeys(cache, PRIMARY_KEYS_NUMBER); for (Integer key : keys) cache.put(key, VAL_PREFIX + key); @@ -189,11 +188,11 @@ protected void runTestGetAllRequestsDistribution(final UUID destId, final boolea assert macs.put(destId, clientMac) != null; - setMacAddress(G.allGrids(), macs); + replaceMacAddresses(G.allGrids(), macs); IgniteCache cache = grid(0).createCache(replicatedCache()); - Set keys = new TreeSet<>(primaryKeys(cache, PRIMARY_KEYS_NUMBER)); + List keys = primaryKeys(cache, PRIMARY_KEYS_NUMBER); for (Integer key : keys) cache.put(key, VAL_PREFIX + key); @@ -210,10 +209,10 @@ protected void runTestGetAllRequestsDistribution(final UUID destId, final boolea * @param keys Keys to get. * @param batchMode Test mode. */ - protected void getAndValidateData(IgniteCache cache, Set keys, boolean batchMode) { + protected void getAndValidateData(IgniteCache cache, List keys, boolean batchMode) { try (Transaction tx = grid(CLIENT_NAME).transactions().txStart()) { if (batchMode) { - Map results = cache.getAll(keys); + Map results = cache.getAll(new TreeSet<>(keys)); for (Map.Entry entry : results.entrySet()) assertEquals(VAL_PREFIX + entry.getKey(), entry.getValue()); @@ -285,7 +284,7 @@ private CacheConfiguration replicatedCache() { * @param instances Started Ignite instances. * @param macs Mapping MAC addresses to UUID. */ - private void setMacAddress(List instances, Map macs) { + private void replaceMacAddresses(List instances, Map macs) { for (Ignite ignite : instances) { for (ClusterNode node : ignite.cluster().nodes()) { String mac = macs.get(node.id()); From 8bf32243aabb02a3dbe3471732fa88bedeba7cd6 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Mon, 12 Mar 2018 17:27:05 +0300 Subject: [PATCH 29/44] ignite-5357: renamed method and variable --- ...eplicatedAtomicCacheGetsDistributionTest.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java index 3f1b9305e2569..244ef4ae0057b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java @@ -139,14 +139,14 @@ protected void runTestGetRequestsGeneratorDistribution(boolean batchMode) throws cache = grid(CLIENT_NAME).getOrCreateCache(CACHE_NAME); - getAndValidateData(cache, keys, batchMode); + validateData(cache, keys, batchMode); for (int i = 0; i < gridCount(); i++) { IgniteEx ignite = grid(i); - long getsCount = ignite.cache(CACHE_NAME).localMetrics().getCacheGets(); + long getsCnt = ignite.cache(CACHE_NAME).localMetrics().getCacheGets(); - assertTrue(getsCount > 0); + assertTrue(getsCnt > 0); } } @@ -199,7 +199,7 @@ protected void runTestGetAllRequestsDistribution(final UUID destId, final boolea cache = grid(CLIENT_NAME).getOrCreateCache(CACHE_NAME); - getAndValidateData(cache, keys, batchMode); + validateData(cache, keys, batchMode); validateRequestsDistribution(destId); } @@ -209,7 +209,7 @@ protected void runTestGetAllRequestsDistribution(final UUID destId, final boolea * @param keys Keys to get. * @param batchMode Test mode. */ - protected void getAndValidateData(IgniteCache cache, List keys, boolean batchMode) { + protected void validateData(IgniteCache cache, List keys, boolean batchMode) { try (Transaction tx = grid(CLIENT_NAME).transactions().txStart()) { if (batchMode) { Map results = cache.getAll(new TreeSet<>(keys)); @@ -233,12 +233,12 @@ protected void validateRequestsDistribution(final UUID destId) { for (int i = 0; i < gridCount(); i++) { IgniteEx ignite = grid(i); - long getsCount = ignite.cache(CACHE_NAME).localMetrics().getCacheGets(); + long getsCnt = ignite.cache(CACHE_NAME).localMetrics().getCacheGets(); if (destId.equals(ignite.localNode().id())) - assertEquals(PRIMARY_KEYS_NUMBER, getsCount); + assertEquals(PRIMARY_KEYS_NUMBER, getsCnt); else - assertEquals(0L, getsCount); + assertEquals(0L, getsCnt); } } From 0b68af07d82e53f8fc223e73dc43c9fa892ba64f Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Mon, 12 Mar 2018 18:35:16 +0300 Subject: [PATCH 30/44] ignite-5357: partitioned cache tests were added --- ...tionedAtomicCacheGetsDistributionTest.java | 48 +++++++++++++++++++ ...alOptimisticCacheGetsDistributionTest.java | 46 ++++++++++++++++++ ...lPessimisticCacheGetsDistributionTest.java | 33 +++++++++++++ ...icatedAtomicCacheGetsDistributionTest.java | 35 ++++++++------ .../IgniteBinaryObjectsTestSuite.java | 7 --- .../testsuites/IgniteCacheTestSuite5.java | 14 ++++++ 6 files changed, 161 insertions(+), 22 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedAtomicCacheGetsDistributionTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedTransactionalOptimisticCacheGetsDistributionTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedTransactionalPessimisticCacheGetsDistributionTest.java diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedAtomicCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedAtomicCacheGetsDistributionTest.java new file mode 100644 index 0000000000000..eee345f6d7d0d --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedAtomicCacheGetsDistributionTest.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.configuration.CacheConfiguration; + +import static org.apache.ignite.cache.CacheMode.PARTITIONED; + +/** + * Tests of partitioned atomic cache's 'get' requests distribution. + */ +public class PartitionedAtomicCacheGetsDistributionTest extends ReplicatedAtomicCacheGetsDistributionTest { + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return PARTITIONED; + } + + /** {@inheritDoc} */ + @Override protected CacheConfiguration cacheConfiguration() { + CacheConfiguration cacheCfg = super.cacheConfiguration(); + cacheCfg.setBackups(backupsCount()); + + return cacheCfg; + } + + /** + * @return Backups count. + */ + protected int backupsCount() { + return gridCount() - 1; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedTransactionalOptimisticCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedTransactionalOptimisticCacheGetsDistributionTest.java new file mode 100644 index 0000000000000..4c882294f89eb --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedTransactionalOptimisticCacheGetsDistributionTest.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; + +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED; + +/** + * Tests of optimistic transactional partitioned cache's 'get' requests distribution. + */ +public class PartitionedTransactionalOptimisticCacheGetsDistributionTest extends PartitionedAtomicCacheGetsDistributionTest { + /** {@inheritDoc} */ + @Override protected CacheAtomicityMode atomicityMode() { + return TRANSACTIONAL; + } + + /** {@inheritDoc} */ + @Override protected TransactionIsolation transactionIsolation() { + return READ_COMMITTED; + } + + /** {@inheritDoc} */ + @Override protected TransactionConcurrency transactionConcurrency() { + return OPTIMISTIC; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedTransactionalPessimisticCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedTransactionalPessimisticCacheGetsDistributionTest.java new file mode 100644 index 0000000000000..78ea7a6f8d30d --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedTransactionalPessimisticCacheGetsDistributionTest.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.transactions.TransactionConcurrency; + +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; + +/** + * Tests of pessimistic transactional partitioned cache's 'get' requests distribution. + */ +public class PartitionedTransactionalPessimisticCacheGetsDistributionTest + extends PartitionedTransactionalOptimisticCacheGetsDistributionTest { + /** {@inheritDoc} */ + @Override protected TransactionConcurrency transactionConcurrency() { + return PESSIMISTIC; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java index 244ef4ae0057b..a5165dc3b0f2a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java @@ -25,6 +25,7 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; @@ -48,7 +49,7 @@ */ public class ReplicatedAtomicCacheGetsDistributionTest extends GridCacheAbstractSelfTest { /** Cache name. */ - private static final String CACHE_NAME = "replicatedCache"; + private static final String CACHE_NAME = "getsDistributionTest"; /** Client nodes instance's name. */ private static final String CLIENT_NAME = "client"; @@ -130,16 +131,14 @@ public void testGetAllRequestsGeneratorDistribution() throws Exception { * @throws Exception In case of an error. */ protected void runTestGetRequestsGeneratorDistribution(boolean batchMode) throws Exception { - IgniteCache cache = grid(0).createCache(replicatedCache()); + IgniteCache cache = grid(0).createCache(cacheConfiguration()); List keys = primaryKeys(cache, PRIMARY_KEYS_NUMBER); for (Integer key : keys) cache.put(key, VAL_PREFIX + key); - cache = grid(CLIENT_NAME).getOrCreateCache(CACHE_NAME); - - validateData(cache, keys, batchMode); + validateData(keys, batchMode); for (int i = 0; i < gridCount(); i++) { IgniteEx ignite = grid(i); @@ -186,30 +185,29 @@ protected void runTestGetAllRequestsDistribution(final UUID destId, final boolea String clientMac = macs.get(grid(CLIENT_NAME).localNode().id()); - assert macs.put(destId, clientMac) != null; + macs.put(destId, clientMac); replaceMacAddresses(G.allGrids(), macs); - IgniteCache cache = grid(0).createCache(replicatedCache()); + IgniteCache cache = grid(0).createCache(cacheConfiguration()); List keys = primaryKeys(cache, PRIMARY_KEYS_NUMBER); for (Integer key : keys) cache.put(key, VAL_PREFIX + key); - cache = grid(CLIENT_NAME).getOrCreateCache(CACHE_NAME); - - validateData(cache, keys, batchMode); + validateData(keys, batchMode); validateRequestsDistribution(destId); } /** - * @param cache Ignite cache. * @param keys Keys to get. * @param batchMode Test mode. */ - protected void validateData(IgniteCache cache, List keys, boolean batchMode) { + protected void validateData(List keys, boolean batchMode) { + IgniteCache cache = grid(CLIENT_NAME).getOrCreateCache(CACHE_NAME); + try (Transaction tx = grid(CLIENT_NAME).transactions().txStart()) { if (batchMode) { Map results = cache.getAll(new TreeSet<>(keys)); @@ -268,14 +266,21 @@ protected TransactionConcurrency transactionConcurrency() { return PESSIMISTIC; } + /** + * @return Caching mode. + */ + protected CacheMode cacheMode() { + return REPLICATED; + } + /** * @return Replicated cache configuration. */ - private CacheConfiguration replicatedCache() { + protected CacheConfiguration cacheConfiguration() { return new CacheConfiguration(CACHE_NAME) - .setCacheMode(REPLICATED) - .setWriteSynchronizationMode(FULL_SYNC) + .setCacheMode(cacheMode()) .setAtomicityMode(atomicityMode()) + .setWriteSynchronizationMode(FULL_SYNC) .setReadFromBackup(true) .setStatisticsEnabled(true); } diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java index 0d5905c605153..c0211be4adbd7 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java @@ -52,9 +52,6 @@ import org.apache.ignite.internal.binary.noncompact.BinaryObjectBuilderNonCompactSimpleNameLowerCaseMappersSelfTest; import org.apache.ignite.internal.binary.streams.BinaryHeapStreamByteOrderSelfTest; import org.apache.ignite.internal.binary.streams.BinaryOffheapStreamByteOrderSelfTest; -import org.apache.ignite.internal.processors.cache.ReplicatedAtomicCacheGetsDistributionTest; -import org.apache.ignite.internal.processors.cache.ReplicatedTransactionalOptimisticCacheGetsDistributionTest; -import org.apache.ignite.internal.processors.cache.ReplicatedTransactionalPessimisticCacheGetsDistributionTest; import org.apache.ignite.internal.processors.cache.binary.BinaryAtomicCacheLocalEntriesSelfTest; import org.apache.ignite.internal.processors.cache.binary.BinaryMetadataUpdatesFlowTest; import org.apache.ignite.internal.processors.cache.binary.BinaryTxCacheLocalEntriesSelfTest; @@ -151,10 +148,6 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridCacheBinaryObjectUserClassloaderSelfTest.class); - suite.addTestSuite(ReplicatedAtomicCacheGetsDistributionTest.class); - suite.addTestSuite(ReplicatedTransactionalOptimisticCacheGetsDistributionTest.class); - suite.addTestSuite(ReplicatedTransactionalPessimisticCacheGetsDistributionTest.class); - return suite; } } diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java index ee57fa7c2e822..6127fa4d1704f 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java @@ -35,7 +35,13 @@ import org.apache.ignite.internal.processors.cache.IgniteCachePutStackOverflowSelfTest; import org.apache.ignite.internal.processors.cache.IgniteCacheReadThroughEvictionsVariationsSuite; import org.apache.ignite.internal.processors.cache.IgniteCacheStoreCollectionTest; +import org.apache.ignite.internal.processors.cache.PartitionedAtomicCacheGetsDistributionTest; +import org.apache.ignite.internal.processors.cache.PartitionedTransactionalOptimisticCacheGetsDistributionTest; +import org.apache.ignite.internal.processors.cache.PartitionedTransactionalPessimisticCacheGetsDistributionTest; import org.apache.ignite.internal.processors.cache.PartitionsExchangeOnDiscoveryHistoryOverflowTest; +import org.apache.ignite.internal.processors.cache.ReplicatedAtomicCacheGetsDistributionTest; +import org.apache.ignite.internal.processors.cache.ReplicatedTransactionalOptimisticCacheGetsDistributionTest; +import org.apache.ignite.internal.processors.cache.ReplicatedTransactionalPessimisticCacheGetsDistributionTest; import org.apache.ignite.internal.processors.cache.distributed.Cache64kPartitionsTest; import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentNodeJoinValidationTest; import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentTest; @@ -101,6 +107,14 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(Cache64kPartitionsTest.class); + suite.addTestSuite(ReplicatedAtomicCacheGetsDistributionTest.class); + suite.addTestSuite(ReplicatedTransactionalOptimisticCacheGetsDistributionTest.class); + suite.addTestSuite(ReplicatedTransactionalPessimisticCacheGetsDistributionTest.class); + + suite.addTestSuite(PartitionedAtomicCacheGetsDistributionTest.class); + suite.addTestSuite(PartitionedTransactionalOptimisticCacheGetsDistributionTest.class); + suite.addTestSuite(PartitionedTransactionalPessimisticCacheGetsDistributionTest.class); + return suite; } } From 04ddc6e65950e626908fdf14588fc8bd7df9eb55 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Mon, 12 Mar 2018 18:40:13 +0300 Subject: [PATCH 31/44] ignite-5357: minor fixes --- ...edTransactionalOptimisticCacheGetsDistributionTest.java | 7 ------- .../cache/ReplicatedAtomicCacheGetsDistributionTest.java | 2 +- ...edTransactionalOptimisticCacheGetsDistributionTest.java | 7 ------- 3 files changed, 1 insertion(+), 15 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedTransactionalOptimisticCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedTransactionalOptimisticCacheGetsDistributionTest.java index 4c882294f89eb..37e92a1adf3d8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedTransactionalOptimisticCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedTransactionalOptimisticCacheGetsDistributionTest.java @@ -19,11 +19,9 @@ import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.transactions.TransactionConcurrency; -import org.apache.ignite.transactions.TransactionIsolation; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC; -import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED; /** * Tests of optimistic transactional partitioned cache's 'get' requests distribution. @@ -34,11 +32,6 @@ public class PartitionedTransactionalOptimisticCacheGetsDistributionTest extends return TRANSACTIONAL; } - /** {@inheritDoc} */ - @Override protected TransactionIsolation transactionIsolation() { - return READ_COMMITTED; - } - /** {@inheritDoc} */ @Override protected TransactionConcurrency transactionConcurrency() { return OPTIMISTIC; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java index a5165dc3b0f2a..63ba9c1156f3d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java @@ -274,7 +274,7 @@ protected CacheMode cacheMode() { } /** - * @return Replicated cache configuration. + * @return Cache configuration. */ protected CacheConfiguration cacheConfiguration() { return new CacheConfiguration(CACHE_NAME) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalOptimisticCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalOptimisticCacheGetsDistributionTest.java index 3bc680972c050..8d8b013e95b9e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalOptimisticCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalOptimisticCacheGetsDistributionTest.java @@ -19,11 +19,9 @@ import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.transactions.TransactionConcurrency; -import org.apache.ignite.transactions.TransactionIsolation; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC; -import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED; /** * Tests of optimistic transactional replicated cache's 'get' requests distribution. @@ -34,11 +32,6 @@ public class ReplicatedTransactionalOptimisticCacheGetsDistributionTest extends return TRANSACTIONAL; } - /** {@inheritDoc} */ - @Override protected TransactionIsolation transactionIsolation() { - return READ_COMMITTED; - } - /** {@inheritDoc} */ @Override protected TransactionConcurrency transactionConcurrency() { return OPTIMISTIC; From cb1c59f54dd41dc63a681257113f33203716bfd0 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Mon, 12 Mar 2018 18:46:12 +0300 Subject: [PATCH 32/44] ignite-5357: returned transactionIsolation in transactional tests --- ...edTransactionalOptimisticCacheGetsDistributionTest.java | 7 +++++++ ...edTransactionalOptimisticCacheGetsDistributionTest.java | 7 +++++++ 2 files changed, 14 insertions(+) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedTransactionalOptimisticCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedTransactionalOptimisticCacheGetsDistributionTest.java index 37e92a1adf3d8..4c882294f89eb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedTransactionalOptimisticCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedTransactionalOptimisticCacheGetsDistributionTest.java @@ -19,9 +19,11 @@ import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED; /** * Tests of optimistic transactional partitioned cache's 'get' requests distribution. @@ -32,6 +34,11 @@ public class PartitionedTransactionalOptimisticCacheGetsDistributionTest extends return TRANSACTIONAL; } + /** {@inheritDoc} */ + @Override protected TransactionIsolation transactionIsolation() { + return READ_COMMITTED; + } + /** {@inheritDoc} */ @Override protected TransactionConcurrency transactionConcurrency() { return OPTIMISTIC; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalOptimisticCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalOptimisticCacheGetsDistributionTest.java index 8d8b013e95b9e..3bc680972c050 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalOptimisticCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedTransactionalOptimisticCacheGetsDistributionTest.java @@ -19,9 +19,11 @@ import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED; /** * Tests of optimistic transactional replicated cache's 'get' requests distribution. @@ -32,6 +34,11 @@ public class ReplicatedTransactionalOptimisticCacheGetsDistributionTest extends return TRANSACTIONAL; } + /** {@inheritDoc} */ + @Override protected TransactionIsolation transactionIsolation() { + return READ_COMMITTED; + } + /** {@inheritDoc} */ @Override protected TransactionConcurrency transactionConcurrency() { return OPTIMISTIC; From 2c3b7f8ce4ac58cad6f6962cfac2ec108e111e86 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Mon, 12 Mar 2018 18:57:17 +0300 Subject: [PATCH 33/44] ignite-5357: polish --- ...tionedAtomicCacheGetsDistributionTest.java | 1 + ...icatedAtomicCacheGetsDistributionTest.java | 21 ++++++++++++------- 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedAtomicCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedAtomicCacheGetsDistributionTest.java index eee345f6d7d0d..2241a956e0908 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedAtomicCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/PartitionedAtomicCacheGetsDistributionTest.java @@ -34,6 +34,7 @@ public class PartitionedAtomicCacheGetsDistributionTest extends ReplicatedAtomic /** {@inheritDoc} */ @Override protected CacheConfiguration cacheConfiguration() { CacheConfiguration cacheCfg = super.cacheConfiguration(); + cacheCfg.setBackups(backupsCount()); return cacheCfg; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java index 63ba9c1156f3d..7b931861cf713 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java @@ -64,7 +64,11 @@ public class ReplicatedAtomicCacheGetsDistributionTest extends GridCacheAbstract @Override protected void beforeTestsStarted() throws Exception { super.beforeTestsStarted(); - startGrid(getConfiguration(CLIENT_NAME).setClientMode(true)); + IgniteConfiguration clientCfg = getConfiguration(CLIENT_NAME); + + clientCfg.setClientMode(true); + + startGrid(clientCfg); } /** {@inheritDoc} */ @@ -277,12 +281,15 @@ protected CacheMode cacheMode() { * @return Cache configuration. */ protected CacheConfiguration cacheConfiguration() { - return new CacheConfiguration(CACHE_NAME) - .setCacheMode(cacheMode()) - .setAtomicityMode(atomicityMode()) - .setWriteSynchronizationMode(FULL_SYNC) - .setReadFromBackup(true) - .setStatisticsEnabled(true); + CacheConfiguration cfg = new CacheConfiguration(CACHE_NAME); + + cfg.setCacheMode(cacheMode()); + cfg.setAtomicityMode(atomicityMode()); + cfg.setWriteSynchronizationMode(FULL_SYNC); + cfg.setReadFromBackup(true); + cfg.setStatisticsEnabled(true); + + return cfg; } /** From 6dcefa4baf7cf1bbb1c1815b56a11e7119bf2a05 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Mon, 19 Mar 2018 15:34:38 +0300 Subject: [PATCH 34/44] ignite-5910: review fixes --- ...icatedAtomicCacheGetsDistributionTest.java | 29 ++++++++++--------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java index 7b931861cf713..a9e8af24bf050 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java @@ -114,27 +114,27 @@ public class ReplicatedAtomicCacheGetsDistributionTest extends GridCacheAbstract * Test 'get' operations requests generator distribution. * * @throws Exception In case of an error. - * @see #runTestGetRequestsGeneratorDistribution(boolean) + * @see #runTestBalancingDistribution(boolean) */ public void testGetRequestsGeneratorDistribution() throws Exception { - runTestGetRequestsGeneratorDistribution(false); + runTestBalancingDistribution(false); } /** * Test 'getAll' operations requests generator distribution. * * @throws Exception In case of an error. - * @see #runTestGetRequestsGeneratorDistribution(boolean) + * @see #runTestBalancingDistribution(boolean) */ public void testGetAllRequestsGeneratorDistribution() throws Exception { - runTestGetRequestsGeneratorDistribution(true); + runTestBalancingDistribution(true); } /** - * @param batchMode Test mode. + * @param batchMode Whenever 'get' or 'getAll' operations are used in the test. * @throws Exception In case of an error. */ - protected void runTestGetRequestsGeneratorDistribution(boolean batchMode) throws Exception { + protected void runTestBalancingDistribution(boolean batchMode) throws Exception { IgniteCache cache = grid(0).createCache(cacheConfiguration()); List keys = primaryKeys(cache, PRIMARY_KEYS_NUMBER); @@ -157,34 +157,34 @@ protected void runTestGetRequestsGeneratorDistribution(boolean batchMode) throws * Tests that the 'get' operation requests are routed to node with same MAC address as at requester. * * @throws Exception In case of an error. - * @see #runTestGetAllRequestsDistribution(UUID, boolean) + * @see #runTestSameHostDistribution(UUID, boolean) */ public void testGetRequestsDistribution() throws Exception { UUID destId = grid(0).localNode().id(); - runTestGetAllRequestsDistribution(destId, false); + runTestSameHostDistribution(destId, false); } /** * Tests that the 'getAll' operation requests are routed to node with same MAC address as at requester. * * @throws Exception In case of an error. - * @see #runTestGetAllRequestsDistribution(UUID, boolean) + * @see #runTestSameHostDistribution(UUID, boolean) */ public void testGetAllRequestsDistribution() throws Exception { UUID destId = grid(gridCount() - 1).localNode().id(); - runTestGetAllRequestsDistribution(destId, true); + runTestSameHostDistribution(destId, true); } /** - * Tests that the 'get' and 'getAll requests are routed to node with same MAC address as at requester. + * Tests that the 'get' and 'getAll' requests are routed to node with same MAC address as at requester. * * @param destId Destination Ignite instance id for requests distribution. * @param batchMode Test mode. * @throws Exception In case of an error. */ - protected void runTestGetAllRequestsDistribution(final UUID destId, final boolean batchMode) throws Exception { + protected void runTestSameHostDistribution(final UUID destId, final boolean batchMode) throws Exception { Map macs = getClusterMacs(); String clientMac = macs.get(grid(CLIENT_NAME).localNode().id()); @@ -273,7 +273,7 @@ protected TransactionConcurrency transactionConcurrency() { /** * @return Caching mode. */ - protected CacheMode cacheMode() { + @Override protected CacheMode cacheMode() { return REPLICATED; } @@ -301,9 +301,10 @@ private void replaceMacAddresses(List instances, Map macs) for (ClusterNode node : ignite.cluster().nodes()) { String mac = macs.get(node.id()); - assert mac != null; + assertNotNull(mac); Map attrs = new HashMap<>(node.attributes()); + attrs.put(ATTR_MACS, mac); ((TcpDiscoveryNode)node).setAttributes(attrs); From 99475a798455b0ad9832bb7ea2c6989e007b908c Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Mon, 19 Mar 2018 19:49:11 +0300 Subject: [PATCH 35/44] ignite-5910: old behavior switcher was added --- .../apache/ignite/IgniteSystemProperties.java | 10 ++++++ .../processors/cache/GridCacheContext.java | 13 ++++++++ .../processors/cache/GridCacheUtils.java | 32 +++++++++++++++++-- .../dht/GridPartitionedGetFuture.java | 2 +- .../dht/GridPartitionedSingleGetFuture.java | 2 +- .../distributed/near/GridNearGetFuture.java | 2 +- 6 files changed, 55 insertions(+), 6 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index b1a1542b3f32b..c4787f466c895 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -25,6 +25,7 @@ import javax.net.ssl.HostnameVerifier; import org.apache.ignite.cache.CacheEntryProcessor; import org.apache.ignite.cluster.ClusterGroup; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.internal.client.GridClient; import org.apache.ignite.internal.marshaller.optimized.OptimizedMarshaller; @@ -821,6 +822,15 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_WAL_FSYNC_WITH_DEDICATED_WORKER = "IGNITE_WAL_FSYNC_WITH_DEDICATED_WORKER"; + /** + * Whenever read load balancing is enabled, that means 'get' requests will be distributed between primary and backup + * nodes if it is possible and {@link CacheConfiguration#readFromBackup} is {@code true}. + * + * Default is {@code true}. + * + * @see CacheConfiguration#readFromBackup + */ + public static final String IGNITE_READ_LOAD_BALANCING = "IGNITE_READ_LOAD_BALANCING"; /** * Enforces singleton. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java index 103a97670c4b5..96067cc48e269 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java @@ -39,6 +39,7 @@ import javax.cache.processor.EntryProcessorResult; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.binary.BinaryField; import org.apache.ignite.binary.BinaryObjectBuilder; import org.apache.ignite.cache.CacheInterceptor; @@ -106,6 +107,7 @@ import org.apache.ignite.plugin.security.SecurityPermission; import org.jetbrains.annotations.Nullable; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_READ_LOAD_BALANCING; import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; @@ -258,6 +260,9 @@ public class GridCacheContext implements Externalizable { /** Statistics enabled flag. */ private volatile boolean statisticsEnabled; + /** Whether to enable read load balancing. */ + private final boolean readLoadBalancingEnabled = IgniteSystemProperties.getBoolean(IGNITE_READ_LOAD_BALANCING, true); + /** * Empty constructor required for {@link Externalizable}. */ @@ -435,6 +440,14 @@ public boolean isReplicatedAffinityNode() { return isReplicated() && affinityNode(); } + /** + * @return {@code true} if read load balancing is enabled. + * @see IgniteSystemProperties#IGNITE_READ_LOAD_BALANCING + */ + public boolean isReadLoadBalancingEnabled() { + return readLoadBalancingEnabled; + } + /** * @throws IgniteCheckedException If failed to wait. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index 8e779123dc3bb..3553aa96072b0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -464,13 +464,17 @@ public static Collection affinityNodes(GridCacheContext ctx, Affini /** * Determines an affinity node to send get request to. * - * @param ctx Context. + * @param ctx Cache context. * @param affNodes All affinity nodes. * @param canRemap Flag indicating that 'get' should be done on a locked topology version. + * @param balancingEnabled Whether read load balancing is enabled. * @return Affinity node to get key from or {@code null} if there is no suitable alive node. */ - @Nullable public static ClusterNode affinityNode(GridCacheContext ctx, List affNodes, - boolean canRemap) { + @Nullable public static ClusterNode selectAffinityNode(GridCacheContext ctx, List affNodes, + boolean canRemap, boolean balancingEnabled) { + if (!balancingEnabled) + return selectAffinityNode(ctx, affNodes, canRemap); + if (!ctx.config().isReadFromBackup()) return affNodes.get(0); @@ -497,6 +501,28 @@ public static Collection affinityNodes(GridCacheContext ctx, Affini return n0; } + /** + * Determines an affinity node to send get request to. + * + * @param ctx Cache context. + * @param affNodes All affinity nodes. + * @param canRemap Flag indicating that 'get' should be done on a locked topology version. + * @return Affinity node to get key from. + */ + @Nullable private static ClusterNode selectAffinityNode(GridCacheContext ctx, List affNodes, + boolean canRemap) { + if (!canRemap) { + for (ClusterNode node : affNodes) { + if (ctx.discovery().alive(node)) + return node; + } + + return null; + } + else + return affNodes.get(0); + } + /** * Checks if near cache is enabled for cache context. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java index 1be56fccac97d..168f63b8da791 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java @@ -407,7 +407,7 @@ private boolean map( } } - ClusterNode node = CU.affinityNode(cctx, affNodes, canRemap); + ClusterNode node = CU.selectAffinityNode(cctx, affNodes, canRemap, cctx.isReadLoadBalancingEnabled()); if (node == null) { onDone(serverNotFoundError(topVer)); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java index 61eee89cd4cdd..4436942155797 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java @@ -347,7 +347,7 @@ private void map(final AffinityTopologyVersion topVer) { } } - ClusterNode affNode = CU.affinityNode(cctx, affNodes, canRemap); + ClusterNode affNode = CU.selectAffinityNode(cctx, affNodes, canRemap, cctx.isReadLoadBalancingEnabled()); if (affNode == null) { onDone(serverNotFoundError(topVer)); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java index 1552132f0c6e9..ea3fb21419dbc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java @@ -490,7 +490,7 @@ private Map map( } } - ClusterNode affNode = CU.affinityNode(cctx, affNodes, canRemap); + ClusterNode affNode = CU.selectAffinityNode(cctx, affNodes, canRemap, cctx.isReadLoadBalancingEnabled()); if (affNode == null) { onDone(serverNotFoundError(topVer)); From 6445ffbb5ccd51df2e1b52f95c65e4d536899b53 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Mon, 19 Mar 2018 20:29:34 +0300 Subject: [PATCH 36/44] ignite-5910: wip --- .../processors/cache/GridCacheContext.java | 49 +++++++++++++++ .../processors/cache/GridCacheUtils.java | 62 ------------------- .../dht/GridPartitionedGetFuture.java | 2 +- .../dht/GridPartitionedSingleGetFuture.java | 2 +- .../distributed/near/GridNearGetFuture.java | 2 +- 5 files changed, 52 insertions(+), 65 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java index 96067cc48e269..a8d6b67ce0139 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java @@ -32,6 +32,7 @@ import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ThreadLocalRandom; import javax.cache.Cache; import javax.cache.configuration.Factory; import javax.cache.expiry.EternalExpiryPolicy; @@ -114,6 +115,7 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC; import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_STARTED; import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_STOPPED; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MACS; import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.OWNING; /** @@ -2170,6 +2172,53 @@ else if (type == EVT_CACHE_REBALANCE_STOPPED) { return true; } + /** + * Determines an affinity node to send get request to. + * + * @param affNodes All affinity nodes. + * @param canRemap Flag indicating that 'get' should be done on a locked topology version. + * @return Affinity node to get key from or {@code null} if there is no suitable alive node. + */ + @Nullable public ClusterNode selectAffinityNode(List affNodes, boolean canRemap) { + if (!readLoadBalancingEnabled) { + if (!canRemap) { + for (ClusterNode node : affNodes) { + if (ctx.discovery().alive(node)) + return node; + } + + return null; + } + else + return affNodes.get(0); + } + + if (!config().isReadFromBackup()) + return affNodes.get(0); + + String locMacs = localNode().attribute(ATTR_MACS); + + assert locMacs != null; + + int r = ThreadLocalRandom.current().nextInt(affNodes.size()); + + ClusterNode n0 = null; + + for (ClusterNode node : affNodes) { + if (canRemap || discovery().alive(node)) { + if (locMacs.equals(node.attribute(ATTR_MACS))) + return node; + + if (r >= 0 || n0 == null) + n0 = node; + } + + r--; + } + + return n0; + } + /** * Prepare affinity field for builder (if possible). * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index 3553aa96072b0..b9fab12fa975f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -461,68 +461,6 @@ public static Collection affinityNodes(GridCacheContext ctx, Affini return ctx.discovery().cacheGroupAffinityNodes(ctx.groupId(), topVer); } - /** - * Determines an affinity node to send get request to. - * - * @param ctx Cache context. - * @param affNodes All affinity nodes. - * @param canRemap Flag indicating that 'get' should be done on a locked topology version. - * @param balancingEnabled Whether read load balancing is enabled. - * @return Affinity node to get key from or {@code null} if there is no suitable alive node. - */ - @Nullable public static ClusterNode selectAffinityNode(GridCacheContext ctx, List affNodes, - boolean canRemap, boolean balancingEnabled) { - if (!balancingEnabled) - return selectAffinityNode(ctx, affNodes, canRemap); - - if (!ctx.config().isReadFromBackup()) - return affNodes.get(0); - - String locMacs = ctx.localNode().attribute(ATTR_MACS); - - assert locMacs != null; - - int r = ThreadLocalRandom.current().nextInt(affNodes.size()); - - ClusterNode n0 = null; - - for (ClusterNode node : affNodes) { - if (canRemap || ctx.discovery().alive(node)) { - if (locMacs.equals(node.attribute(ATTR_MACS))) - return node; - - if (r >= 0 || n0 == null) - n0 = node; - } - - r--; - } - - return n0; - } - - /** - * Determines an affinity node to send get request to. - * - * @param ctx Cache context. - * @param affNodes All affinity nodes. - * @param canRemap Flag indicating that 'get' should be done on a locked topology version. - * @return Affinity node to get key from. - */ - @Nullable private static ClusterNode selectAffinityNode(GridCacheContext ctx, List affNodes, - boolean canRemap) { - if (!canRemap) { - for (ClusterNode node : affNodes) { - if (ctx.discovery().alive(node)) - return node; - } - - return null; - } - else - return affNodes.get(0); - } - /** * Checks if near cache is enabled for cache context. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java index 168f63b8da791..cceb4bed90350 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java @@ -407,7 +407,7 @@ private boolean map( } } - ClusterNode node = CU.selectAffinityNode(cctx, affNodes, canRemap, cctx.isReadLoadBalancingEnabled()); + ClusterNode node = cctx.selectAffinityNode(affNodes, canRemap); if (node == null) { onDone(serverNotFoundError(topVer)); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java index 4436942155797..be5dac4dbcbad 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java @@ -347,7 +347,7 @@ private void map(final AffinityTopologyVersion topVer) { } } - ClusterNode affNode = CU.selectAffinityNode(cctx, affNodes, canRemap, cctx.isReadLoadBalancingEnabled()); + ClusterNode affNode = cctx.selectAffinityNode(affNodes, canRemap); if (affNode == null) { onDone(serverNotFoundError(topVer)); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java index ea3fb21419dbc..bcc38cbb1ac54 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java @@ -490,7 +490,7 @@ private Map map( } } - ClusterNode affNode = CU.selectAffinityNode(cctx, affNodes, canRemap, cctx.isReadLoadBalancingEnabled()); + ClusterNode affNode = cctx.selectAffinityNode(affNodes, canRemap); if (affNode == null) { onDone(serverNotFoundError(topVer)); From 5ea66145ae5c976e0dad5ecde708771601788af8 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Mon, 19 Mar 2018 20:34:25 +0300 Subject: [PATCH 37/44] ignite-5910: wip2 --- .../internal/processors/cache/GridCacheContext.java | 12 ++---------- .../distributed/dht/GridPartitionedGetFuture.java | 2 +- .../dht/GridPartitionedSingleGetFuture.java | 2 +- .../cache/distributed/near/GridNearGetFuture.java | 2 +- 4 files changed, 5 insertions(+), 13 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java index a8d6b67ce0139..fda3fa516d6a6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java @@ -441,15 +441,7 @@ public boolean affinityNode() { public boolean isReplicatedAffinityNode() { return isReplicated() && affinityNode(); } - - /** - * @return {@code true} if read load balancing is enabled. - * @see IgniteSystemProperties#IGNITE_READ_LOAD_BALANCING - */ - public boolean isReadLoadBalancingEnabled() { - return readLoadBalancingEnabled; - } - + /** * @throws IgniteCheckedException If failed to wait. */ @@ -2179,7 +2171,7 @@ else if (type == EVT_CACHE_REBALANCE_STOPPED) { * @param canRemap Flag indicating that 'get' should be done on a locked topology version. * @return Affinity node to get key from or {@code null} if there is no suitable alive node. */ - @Nullable public ClusterNode selectAffinityNode(List affNodes, boolean canRemap) { + @Nullable public ClusterNode selectAffinityNodeBalanced(List affNodes, boolean canRemap) { if (!readLoadBalancingEnabled) { if (!canRemap) { for (ClusterNode node : affNodes) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java index cceb4bed90350..204a0cecf4729 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java @@ -407,7 +407,7 @@ private boolean map( } } - ClusterNode node = cctx.selectAffinityNode(affNodes, canRemap); + ClusterNode node = cctx.selectAffinityNodeBalanced(affNodes, canRemap); if (node == null) { onDone(serverNotFoundError(topVer)); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java index be5dac4dbcbad..e0aea9a7d1b1d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java @@ -347,7 +347,7 @@ private void map(final AffinityTopologyVersion topVer) { } } - ClusterNode affNode = cctx.selectAffinityNode(affNodes, canRemap); + ClusterNode affNode = cctx.selectAffinityNodeBalanced(affNodes, canRemap); if (affNode == null) { onDone(serverNotFoundError(topVer)); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java index bcc38cbb1ac54..b35f524bb735b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java @@ -490,7 +490,7 @@ private Map map( } } - ClusterNode affNode = cctx.selectAffinityNode(affNodes, canRemap); + ClusterNode affNode = cctx.selectAffinityNodeBalanced(affNodes, canRemap); if (affNode == null) { onDone(serverNotFoundError(topVer)); From 86cebff6e045e50a705049d0194f769450320154 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Mon, 19 Mar 2018 20:36:36 +0300 Subject: [PATCH 38/44] cleanup --- .../apache/ignite/internal/processors/cache/GridCacheUtils.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index b9fab12fa975f..0926a70cc2ae3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -31,7 +31,6 @@ import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicBoolean; import javax.cache.Cache; import javax.cache.CacheException; @@ -117,7 +116,6 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC; import static org.apache.ignite.configuration.CacheConfiguration.DFLT_CACHE_MODE; import static org.apache.ignite.internal.GridTopic.TOPIC_REPLICATION; -import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MACS; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.READ; /** From 4ff71d0b542e981fd95eb59b667f848baa587031 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Mon, 19 Mar 2018 20:38:47 +0300 Subject: [PATCH 39/44] cleanup --- .../ignite/internal/processors/cache/GridCacheContext.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java index fda3fa516d6a6..820a99f3739c0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java @@ -441,7 +441,7 @@ public boolean affinityNode() { public boolean isReplicatedAffinityNode() { return isReplicated() && affinityNode(); } - + /** * @throws IgniteCheckedException If failed to wait. */ From 6df095f734135cf528e92e75a478f53a2114e0ee Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Mon, 19 Mar 2018 20:58:45 +0300 Subject: [PATCH 40/44] ignite-5357: locMacs moved to classes field --- .../ignite/internal/processors/cache/GridCacheContext.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java index 820a99f3739c0..adbee2c081408 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java @@ -265,6 +265,9 @@ public class GridCacheContext implements Externalizable { /** Whether to enable read load balancing. */ private final boolean readLoadBalancingEnabled = IgniteSystemProperties.getBoolean(IGNITE_READ_LOAD_BALANCING, true); + /** Local node's MAC address. */ + private String locMacs; + /** * Empty constructor required for {@link Externalizable}. */ @@ -384,6 +387,8 @@ public GridCacheContext( expiryPlc = null; itHolder = new CacheWeakQueryIteratorsHolder(log); + + locMacs = localNode().attribute(ATTR_MACS); } /** @@ -2188,8 +2193,6 @@ else if (type == EVT_CACHE_REBALANCE_STOPPED) { if (!config().isReadFromBackup()) return affNodes.get(0); - String locMacs = localNode().attribute(ATTR_MACS); - assert locMacs != null; int r = ThreadLocalRandom.current().nextInt(affNodes.size()); From eee4afc5363eb999f09b39f83b2ed10f8463eb8c Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Tue, 20 Mar 2018 15:20:08 +0300 Subject: [PATCH 41/44] ignite-5357: locMacs nonNull assertion was added --- .../ignite/internal/processors/cache/GridCacheContext.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java index adbee2c081408..029bdb9444a1b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java @@ -389,6 +389,8 @@ public GridCacheContext( itHolder = new CacheWeakQueryIteratorsHolder(log); locMacs = localNode().attribute(ATTR_MACS); + + assert locMacs != null; } /** From 2257a45dacde0560a68dfc977560742d94721f33 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Tue, 20 Mar 2018 15:33:31 +0300 Subject: [PATCH 42/44] ignite-5357: readFromBackup variable was added --- .../ignite/internal/processors/cache/GridCacheContext.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java index 029bdb9444a1b..9eb434055f086 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java @@ -265,6 +265,9 @@ public class GridCacheContext implements Externalizable { /** Whether to enable read load balancing. */ private final boolean readLoadBalancingEnabled = IgniteSystemProperties.getBoolean(IGNITE_READ_LOAD_BALANCING, true); + /** Flag indicating whether data can be read from backup. */ + private boolean readFromBackup = CacheConfiguration.DFLT_READ_FROM_BACKUP; + /** Local node's MAC address. */ private String locMacs; @@ -388,6 +391,8 @@ public GridCacheContext( itHolder = new CacheWeakQueryIteratorsHolder(log); + readFromBackup = cacheCfg.isReadFromBackup(); + locMacs = localNode().attribute(ATTR_MACS); assert locMacs != null; @@ -2192,7 +2197,7 @@ else if (type == EVT_CACHE_REBALANCE_STOPPED) { return affNodes.get(0); } - if (!config().isReadFromBackup()) + if (!readFromBackup) return affNodes.get(0); assert locMacs != null; From a67e6b402d17cc8de04cbba6507f2559393623e8 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Tue, 20 Mar 2018 16:34:40 +0300 Subject: [PATCH 43/44] ignite-5357: cleanup --- .../src/main/java/org/apache/ignite/IgniteSystemProperties.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index e21e17d219025..1f67f81d70194 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -830,7 +830,7 @@ public final class IgniteSystemProperties { * Default is {@code true}. */ public static final String IGNITE_LOADED_PAGES_BACKWARD_SHIFT_MAP = "IGNITE_LOADED_PAGES_BACKWARD_SHIFT_MAP"; - + /** * Whenever read load balancing is enabled, that means 'get' requests will be distributed between primary and backup * nodes if it is possible and {@link CacheConfiguration#readFromBackup} is {@code true}. From 5d93e97cbc389a444fb2ac91a3c71f02b73cb700 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Wed, 21 Mar 2018 18:21:59 +0300 Subject: [PATCH 44/44] ignite-5357: refactoring in tests --- ...icatedAtomicCacheGetsDistributionTest.java | 79 +++++++++++++------ 1 file changed, 55 insertions(+), 24 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java index a9e8af24bf050..1d0c6de9aebdb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ReplicatedAtomicCacheGetsDistributionTest.java @@ -20,6 +20,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.TreeSet; import java.util.UUID; import org.apache.ignite.Ignite; @@ -31,8 +32,10 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.TransactionConfiguration; import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; +import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.transactions.Transaction; import org.apache.ignite.transactions.TransactionConcurrency; import org.apache.ignite.transactions.TransactionIsolation; @@ -142,15 +145,58 @@ protected void runTestBalancingDistribution(boolean batchMode) throws Exception for (Integer key : keys) cache.put(key, VAL_PREFIX + key); - validateData(keys, batchMode); + IgniteCache clientCache = grid(CLIENT_NAME).getOrCreateCache(CACHE_NAME); - for (int i = 0; i < gridCount(); i++) { - IgniteEx ignite = grid(i); + assertTrue(GridTestUtils.waitForCondition( + new GridAbsPredicate() { + int batchSize = 10; + int idx = 0; - long getsCnt = ignite.cache(CACHE_NAME).localMetrics().getCacheGets(); + @Override public boolean apply() { + if (idx >= PRIMARY_KEYS_NUMBER) + idx = 0; - assertTrue(getsCnt > 0); - } + try (Transaction tx = grid(CLIENT_NAME).transactions().txStart()) { + if (batchMode) { + Set keys0 = new TreeSet<>(); + + for (int i = idx; i < idx + batchSize && i < PRIMARY_KEYS_NUMBER; i++) + keys0.add(keys.get(i)); + + idx += batchSize; + + Map results = clientCache.getAll(keys0); + + for (Map.Entry entry : results.entrySet()) + assertEquals(VAL_PREFIX + entry.getKey(), entry.getValue()); + } + else { + for (int i = idx; i < idx + gridCount() && i < PRIMARY_KEYS_NUMBER; i++) { + Integer key = keys.get(i); + + assertEquals(VAL_PREFIX + key, clientCache.get(key)); + } + + idx += gridCount(); + } + + tx.commit(); + } + + for (int i = 0; i < gridCount(); i++) { + IgniteEx ignite = grid(i); + + long getsCnt = ignite.cache(CACHE_NAME).localMetrics().getCacheGets(); + + if (getsCnt == 0) + return false; + } + + return true; + } + }, + getTestTimeout()) + ); } /** @@ -200,38 +246,23 @@ protected void runTestSameHostDistribution(final UUID destId, final boolean batc for (Integer key : keys) cache.put(key, VAL_PREFIX + key); - validateData(keys, batchMode); - - validateRequestsDistribution(destId); - } - - /** - * @param keys Keys to get. - * @param batchMode Test mode. - */ - protected void validateData(List keys, boolean batchMode) { - IgniteCache cache = grid(CLIENT_NAME).getOrCreateCache(CACHE_NAME); + IgniteCache clientCache = grid(CLIENT_NAME).getOrCreateCache(CACHE_NAME); try (Transaction tx = grid(CLIENT_NAME).transactions().txStart()) { if (batchMode) { - Map results = cache.getAll(new TreeSet<>(keys)); + Map results = clientCache.getAll(new TreeSet<>(keys)); for (Map.Entry entry : results.entrySet()) assertEquals(VAL_PREFIX + entry.getKey(), entry.getValue()); } else { for (Integer key : keys) - assertEquals(VAL_PREFIX + key, cache.get(key)); + assertEquals(VAL_PREFIX + key, clientCache.get(key)); } tx.commit(); } - } - /** - * @param destId Destination node id. - */ - protected void validateRequestsDistribution(final UUID destId) { for (int i = 0; i < gridCount(); i++) { IgniteEx ignite = grid(i);