You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2016/12/09 11:27:23 UTC
[01/25] ignite git commit: ignite-4296 Optimize
GridDhtPartitionsSingleMessage processing - optimized processing of
GridDhtPartitionsSingleMessage - minor optimizations for
RendezvousAffinityFunction - fixed heartbeats sending in tcp discovery
Repository: ignite
Updated Branches:
refs/heads/ignite-4371 697e2fb81 -> 37a76190d
http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java
deleted file mode 100644
index 377a55f..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java
+++ /dev/null
@@ -1,232 +0,0 @@
-/*
- * 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.distributed.dht;
-
-import java.lang.reflect.Field;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.cache.affinity.Affinity;
-import org.apache.ignite.cache.affinity.AffinityFunction;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
-import org.apache.ignite.internal.processors.cache.GridCacheConcurrentMap;
-import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
-import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap2;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.P1;
-import org.apache.ignite.internal.util.typedef.internal.CU;
-
-import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.OWNING;
-
-/**
- * Utility methods for dht preloader testing.
- */
-public class GridCacheDhtTestUtils {
- /**
- * Ensure singleton.
- */
- private GridCacheDhtTestUtils() {
- // No-op.
- }
-
- /**
- * @param dht Cache.
- * @param keyCnt Number of test keys to put into cache.
- * @throws IgniteCheckedException If failed to prepare.
- */
- @SuppressWarnings({"UnusedAssignment", "unchecked"})
- static void prepareKeys(GridDhtCache<Integer, String> dht, int keyCnt) throws IgniteCheckedException {
- AffinityFunction aff = dht.context().config().getAffinity();
-
- GridCacheConcurrentMap cacheMap;
-
- try {
- Field field = GridCacheAdapter.class.getDeclaredField("map");
-
- field.setAccessible(true);
-
- cacheMap = (GridCacheConcurrentMap)field.get(dht);
- }
- catch (Exception e) {
- throw new IgniteCheckedException("Failed to get cache map.", e);
- }
-
- GridDhtPartitionTopology top = dht.topology();
-
- GridCacheContext ctx = dht.context();
-
- for (int i = 0; i < keyCnt; i++) {
- KeyCacheObject cacheKey = ctx.toCacheKeyObject(i);
-
- cacheMap.putEntryIfObsoleteOrAbsent(
- AffinityTopologyVersion.NONE,
- cacheKey,
- ctx.toCacheKeyObject("value" + i),
- false,
- false);
-
- dht.preloader().request(Collections.singleton(cacheKey), AffinityTopologyVersion.NONE);
-
- GridDhtLocalPartition part = top.localPartition(aff.partition(i), false);
-
- assert part != null;
-
- part.own();
- }
- }
-
- /**
- * @param dht Dht cache.
- * @param idx Cache index
- */
- static void printDhtTopology(GridDhtCache<Integer, String> dht, int idx) {
- final Affinity<Integer> aff = dht.affinity();
-
- Ignite ignite = dht.context().grid();
- ClusterNode locNode = ignite.cluster().localNode();
-
- GridDhtPartitionTopology top = dht.topology();
-
- System.out.println("\nTopology of cache #" + idx + " (" + locNode.id() + ")" + ":");
- System.out.println("----------------------------------");
-
- List<Integer> affParts = new LinkedList<>();
-
- GridDhtPartitionMap2 map = dht.topology().partitions(locNode.id());
-
- if (map != null)
- for (int p : map.keySet())
- affParts.add(p);
-
- Collections.sort(affParts);
-
- System.out.println("Affinity partitions: " + affParts + "\n");
-
- List<GridDhtLocalPartition> locals = new ArrayList<GridDhtLocalPartition>(top.localPartitions());
-
- Collections.sort(locals);
-
- for (final GridDhtLocalPartition part : locals) {
- Collection<ClusterNode> partNodes = aff.mapKeyToPrimaryAndBackups(part.id());
-
- String ownStr = !partNodes.contains(dht.context().localNode()) ? "NOT AN OWNER" :
- F.eqNodes(CU.primary(partNodes), locNode) ? "PRIMARY" : "BACKUP";
-
- Collection<Integer> keys = F.viewReadOnly(dht.keySet(), F.<Integer>identity(), new P1<Integer>() {
- @Override public boolean apply(Integer k) {
- return aff.partition(k) == part.id();
- }
- });
-
- System.out.println("Local partition: [" + part + "], [owning=" + ownStr + ", keyCnt=" + keys.size() +
- ", keys=" + keys + "]");
- }
-
- System.out.println("\nNode map:");
-
- for (Map.Entry<UUID, GridDhtPartitionMap2> e : top.partitionMap(false).entrySet()) {
- List<Integer> list = new ArrayList<>(e.getValue().keySet());
-
- Collections.sort(list);
-
- System.out.println("[node=" + e.getKey() + ", parts=" + list + "]");
- }
-
- System.out.println("");
- }
-
- /**
- * Checks consistency of partitioned cache.
- * Any preload processes must be finished before this method call().
- *
- * @param dht Dht cache.
- * @param idx Cache index.
- * @param log Logger.
- */
- @SuppressWarnings("unchecked")
- static void checkDhtTopology(GridDhtCache<Integer, String> dht, int idx, IgniteLogger log) {
- assert dht != null;
- assert idx >= 0;
- assert log != null;
-
- log.info("Checking balanced state of cache #" + idx);
-
- Affinity<Object> aff = (Affinity)dht.affinity();
-
- Ignite ignite = dht.context().grid();
- ClusterNode locNode = ignite.cluster().localNode();
-
- GridDhtPartitionTopology top = dht.topology();
-
- // Expected partitions calculated with affinity function.
- // They should be in topology in OWNING state.
- Collection<Integer> affParts = new HashSet<>();
-
- GridDhtPartitionMap2 map = dht.topology().partitions(locNode.id());
-
- if (map != null)
- for (int p : map.keySet())
- affParts.add(p);
-
- if (F.isEmpty(affParts))
- return;
-
- for (int p : affParts)
- assert top.localPartition(p, false) != null :
- "Partition does not exist in topology: [cache=" + idx + ", part=" + p + "]";
-
- for (GridDhtLocalPartition p : top.localPartitions()) {
- assert affParts.contains(p.id()) :
- "Invalid local partition: [cache=" + idx + ", part=" + p + ", node partitions=" + affParts + "]";
-
- assert p.state() == OWNING : "Invalid partition state [cache=" + idx + ", part=" + p + "]";
-
- Collection<ClusterNode> partNodes = aff.mapPartitionToPrimaryAndBackups(p.id());
-
- assert partNodes.contains(locNode) :
- "Partition affinity nodes does not contain local node: [cache=" + idx + "]";
- }
-
- // Check keys.
- for (GridCacheEntryEx e : dht.entries()) {
- GridDhtCacheEntry entry = (GridDhtCacheEntry)e;
-
- if (!affParts.contains(entry.partition()))
- log.warning("Partition of stored entry is obsolete for node: [cache=" + idx + ", entry=" + entry +
- ", node partitions=" + affParts + "]");
-
- int p = aff.partition(entry.key());
-
- if (!affParts.contains(p))
- log.warning("Calculated entry partition is not in node partitions: [cache=" + idx + ", part=" + p +
- ", entry=" + entry + ", node partitions=" + affParts + "]");
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 48567da..40b11973 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -58,8 +58,7 @@ import org.apache.ignite.internal.managers.communication.GridMessageListener;
import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap2;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable;
import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery;
import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
@@ -383,17 +382,10 @@ public class GridReduceQueryExecutor {
/**
* @param cctx Cache context.
- * @return {@code true} If cache context
+ * @return {@code True} If cache has partitions in {@link GridDhtPartitionState#MOVING} state.
*/
private boolean hasMovingPartitions(GridCacheContext<?, ?> cctx) {
- GridDhtPartitionFullMap fullMap = cctx.topology().partitionMap(false);
-
- for (GridDhtPartitionMap2 map : fullMap.values()) {
- if (map.hasMovingPartitions())
- return true;
- }
-
- return false;
+ return cctx.topology().hasMovingPartitions();
}
/**
[12/25] ignite git commit: IGNITE-4336 Manual rebalance can't be
requested twice
Posted by sb...@apache.org.
IGNITE-4336 Manual rebalance can't be requested twice
Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/9a691c4f
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/9a691c4f
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/9a691c4f
Branch: refs/heads/ignite-4371
Commit: 9a691c4f6a709caab18ca1442217829b6f578e56
Parents: 59e6fec
Author: Ilya Lantukh <il...@gridgain.com>
Authored: Thu Dec 8 17:58:54 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Dec 8 18:02:46 2016 +0300
----------------------------------------------------------------------
.../processors/cache/GridCacheAdapter.java | 6 +-
.../GridCachePartitionExchangeManager.java | 11 ++-
.../processors/cache/GridCachePreloader.java | 8 ++-
.../cache/GridCachePreloaderAdapter.java | 12 ++--
.../processors/cache/IgniteCacheProxy.java | 4 +-
.../dht/preloader/GridDhtPartitionDemander.java | 46 ++++++++++--
.../GridDhtPartitionsExchangeFuture.java | 16 ++++-
.../dht/preloader/GridDhtPreloader.java | 11 +--
.../cache/CacheRebalancingSelfTest.java | 75 ++++++++++++++++++++
.../testsuites/IgniteCacheTestSuite5.java | 9 ++-
10 files changed, 166 insertions(+), 32 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/9a691c4f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 2e24e67..0f39c69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -84,8 +84,8 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
import org.apache.ignite.internal.processors.cache.affinity.GridCacheAffinityImpl;
import org.apache.ignite.internal.processors.cache.distributed.IgniteExternalizableExpiryPolicy;
import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo;
import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter;
@@ -4651,9 +4651,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
/** {@inheritDoc} */
@Override public IgniteInternalFuture<?> rebalance() {
- ctx.preloader().forcePreload();
-
- return ctx.preloader().syncFuture();
+ return ctx.preloader().forceRebalance();
}
/** {@inheritDoc} */
http://git-wip-us.apache.org/repos/asf/ignite/blob/9a691c4f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index f04a6ce..8ea2169 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -711,9 +711,13 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
*
* @param exchFut Exchange future.
*/
- public void forcePreloadExchange(GridDhtPartitionsExchangeFuture exchFut) {
+ public IgniteInternalFuture<Boolean> forceRebalance(GridDhtPartitionsExchangeFuture exchFut) {
+ GridFutureAdapter<Boolean> fut = new GridFutureAdapter<>();
+
exchWorker.addFuture(
- new GridDhtPartitionsExchangeFuture(cctx, exchFut.discoveryEvent(), exchFut.exchangeId()));
+ new GridDhtPartitionsExchangeFuture(cctx, exchFut.discoveryEvent(), exchFut.exchangeId(), fut));
+
+ return fut;
}
/**
@@ -1771,7 +1775,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
Runnable cur = cacheCtx.preloader().addAssignments(assigns,
forcePreload,
cnt,
- r);
+ r,
+ exchFut.forcedRebalanceFuture());
if (cur != null) {
rebList.add(U.maskName(cacheCtx.name()));
http://git-wip-us.apache.org/repos/asf/ignite/blob/9a691c4f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
index 3c4456d..0c28691 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessageV2;
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
import org.apache.ignite.lang.IgnitePredicate;
import org.jetbrains.annotations.Nullable;
@@ -90,7 +91,8 @@ public interface GridCachePreloader {
public Runnable addAssignments(GridDhtPreloaderAssignments assignments,
boolean forcePreload,
int cnt,
- Runnable next);
+ Runnable next,
+ @Nullable GridFutureAdapter<Boolean> forcedRebFut);
/**
* @param p Preload predicate.
@@ -150,9 +152,9 @@ public interface GridCachePreloader {
AffinityTopologyVersion topVer);
/**
- * Force preload process.
+ * Force Rebalance process.
*/
- public void forcePreload();
+ public IgniteInternalFuture<Boolean> forceRebalance();
/**
* Unwinds undeploys.
http://git-wip-us.apache.org/repos/asf/ignite/blob/9a691c4f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
index 656a960..d7ec288 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments;
import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
import org.apache.ignite.lang.IgnitePredicate;
import org.jetbrains.annotations.Nullable;
@@ -88,8 +89,8 @@ public class GridCachePreloaderAdapter implements GridCachePreloader {
}
/** {@inheritDoc} */
- @Override public void forcePreload() {
- // No-op.
+ @Override public IgniteInternalFuture<Boolean> forceRebalance() {
+ return new GridFinishedFuture<>(true);
}
/** {@inheritDoc} */
@@ -165,8 +166,11 @@ public class GridCachePreloaderAdapter implements GridCachePreloader {
}
/** {@inheritDoc} */
- @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload,
- int cnt, Runnable next) {
+ @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments,
+ boolean forcePreload,
+ int cnt,
+ Runnable next,
+ @Nullable GridFutureAdapter<Boolean> forcedRebFut) {
return null;
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/9a691c4f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index f87fa1d..f03a3b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -2326,9 +2326,7 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
/** {@inheritDoc} */
@Override public IgniteFuture<?> rebalance() {
- ctx.preloader().forcePreload();
-
- return new IgniteFutureImpl<>(ctx.preloader().syncFuture());
+ return new IgniteFutureImpl<>(ctx.preloader().forceRebalance());
}
/**
http://git-wip-us.apache.org/repos/asf/ignite/blob/9a691c4f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index a6808c7..02c31da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -55,6 +55,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartit
import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter;
import org.apache.ignite.internal.util.GridLeanSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
import org.apache.ignite.internal.util.future.GridFutureAdapter;
import org.apache.ignite.internal.util.tostring.GridToStringExclude;
import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -64,6 +65,7 @@ import org.apache.ignite.internal.util.typedef.T2;
import org.apache.ignite.internal.util.typedef.internal.LT;
import org.apache.ignite.internal.util.typedef.internal.S;
import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
import org.apache.ignite.lang.IgnitePredicate;
import org.apache.ignite.lang.IgniteUuid;
import org.apache.ignite.spi.IgniteSpiException;
@@ -216,9 +218,9 @@ public class GridDhtPartitionDemander {
}
/**
- * Force preload.
+ * Force Rebalance.
*/
- void forcePreload() {
+ IgniteInternalFuture<Boolean> forceRebalance() {
GridTimeoutObject obj = lastTimeoutObj.getAndSet(null);
if (obj != null)
@@ -230,14 +232,31 @@ public class GridDhtPartitionDemander {
if (log.isDebugEnabled())
log.debug("Forcing rebalance event for future: " + exchFut);
+ final GridFutureAdapter<Boolean> fut = new GridFutureAdapter<>();
+
exchFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
@Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
- cctx.shared().exchange().forcePreloadExchange(exchFut);
+ IgniteInternalFuture<Boolean> fut0 = cctx.shared().exchange().forceRebalance(exchFut);
+
+ fut0.listen(new IgniteInClosure<IgniteInternalFuture<Boolean>>() {
+ @Override public void apply(IgniteInternalFuture<Boolean> future) {
+ try {
+ fut.onDone(future.get());
+ }
+ catch (Exception e) {
+ fut.onDone(e);
+ }
+ }
+ });
}
});
+
+ return fut;
}
else if (log.isDebugEnabled())
log.debug("Ignoring force rebalance request (no topology event happened yet).");
+
+ return new GridFinishedFuture<>(true);
}
/**
@@ -275,15 +294,19 @@ public class GridDhtPartitionDemander {
* @param force {@code True} if dummy reassign.
* @param cnt Counter.
* @param next Runnable responsible for cache rebalancing start.
+ * @param forcedRebFut External future for forced rebalance.
* @return Rebalancing runnable.
*/
Runnable addAssignments(final GridDhtPreloaderAssignments assigns,
boolean force,
int cnt,
- final Runnable next) {
+ final Runnable next,
+ @Nullable final GridFutureAdapter<Boolean> forcedRebFut) {
if (log.isDebugEnabled())
log.debug("Adding partition assignments: " + assigns);
+ assert force == (forcedRebFut != null);
+
long delay = cctx.config().getRebalanceDelay();
if (delay == 0 || force) {
@@ -301,6 +324,19 @@ public class GridDhtPartitionDemander {
});
}
+ if (forcedRebFut != null) {
+ fut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
+ @Override public void apply(IgniteInternalFuture<Boolean> future) {
+ try {
+ forcedRebFut.onDone(future.get());
+ }
+ catch (Exception e) {
+ forcedRebFut.onDone(e);
+ }
+ }
+ });
+ }
+
rebalanceFut = fut;
fut.sendRebalanceStartedEvent();
@@ -383,7 +419,7 @@ public class GridDhtPartitionDemander {
@Override public void onTimeout() {
exchFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
@Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> f) {
- cctx.shared().exchange().forcePreloadExchange(exchFut);
+ cctx.shared().exchange().forceRebalance(exchFut);
}
});
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/9a691c4f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index e945de9..4f34401 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -54,7 +54,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext;
import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
@@ -65,7 +64,6 @@ import org.apache.ignite.internal.util.tostring.GridToStringExclude;
import org.apache.ignite.internal.util.tostring.GridToStringInclude;
import org.apache.ignite.internal.util.typedef.CI1;
import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.T2;
import org.apache.ignite.internal.util.typedef.internal.CU;
import org.apache.ignite.internal.util.typedef.internal.LT;
import org.apache.ignite.internal.util.typedef.internal.S;
@@ -194,6 +192,9 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
/** */
private boolean centralizedAff;
+ /** Forced Rebalance future. */
+ private GridFutureAdapter<Boolean> forcedRebFut;
+
/**
* Dummy future created to trigger reassignments if partition
* topology changed while preloading.
@@ -227,15 +228,17 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
* @param cctx Cache context.
* @param discoEvt Discovery event.
* @param exchId Exchange id.
+ * @param forcedRebFut Forced Rebalance future.
*/
public GridDhtPartitionsExchangeFuture(GridCacheSharedContext cctx, DiscoveryEvent discoEvt,
- GridDhtPartitionExchangeId exchId) {
+ GridDhtPartitionExchangeId exchId, GridFutureAdapter<Boolean> forcedRebFut) {
dummy = false;
forcePreload = true;
this.exchId = exchId;
this.discoEvt = discoEvt;
this.cctx = cctx;
+ this.forcedRebFut = forcedRebFut;
reassign = true;
@@ -398,6 +401,13 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
}
/**
+ * @return Forced Rebalance future.
+ */
+ @Nullable public GridFutureAdapter<Boolean> forcedRebalanceFuture() {
+ return forcedRebFut;
+ }
+
+ /**
* @return {@code true} if entered to busy state.
*/
private boolean enterBusy() {
http://git-wip-us.apache.org/repos/asf/ignite/blob/9a691c4f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
index 692e7c0..41bc2fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
@@ -413,8 +413,11 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
/** {@inheritDoc} */
@Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments,
- boolean forcePreload, int cnt, Runnable next) {
- return demander.addAssignments(assignments, forcePreload, cnt, next);
+ boolean forceRebalance,
+ int cnt,
+ Runnable next,
+ @Nullable GridFutureAdapter<Boolean> forcedRebFut) {
+ return demander.addAssignments(assignments, forceRebalance, cnt, next, forcedRebFut);
}
/**
@@ -728,8 +731,8 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
}
/** {@inheritDoc} */
- @Override public void forcePreload() {
- demander.forcePreload();
+ @Override public IgniteInternalFuture<Boolean> forceRebalance() {
+ return demander.forceRebalance();
}
/** {@inheritDoc} */
http://git-wip-us.apache.org/repos/asf/ignite/blob/9a691c4f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java
new file mode 100644
index 0000000..8d1f67a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Test for rebalancing.
+ */
+public class CacheRebalancingSelfTest extends GridCommonAbstractTest {
+ /** {@inheritDoc} */
+ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+ IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+ cfg.setCacheConfiguration(new CacheConfiguration());
+
+ return cfg;
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testRebalanceFuture() throws Exception {
+ IgniteEx ignite0 = startGrid(0);
+ startGrid(1);
+
+ IgniteCache<Object, Object> cache = ignite0.cache(null);
+
+ IgniteFuture fut1 = cache.rebalance();
+
+ fut1.get();
+
+ startGrid(2);
+
+ IgniteFuture fut2 = cache.rebalance();
+
+ assert internalFuture(fut2) != internalFuture(fut1);
+
+ fut2.get();
+ }
+
+ /**
+ * @param future Future.
+ * @return Internal future.
+ */
+ private static IgniteInternalFuture internalFuture(IgniteFuture future) {
+ assert future instanceof IgniteFutureImpl;
+
+ return ((IgniteFutureImpl)future).internalFuture();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/9a691c4f/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
----------------------------------------------------------------------
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 7f0e23c..144aac6 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
@@ -19,18 +19,19 @@ package org.apache.ignite.testsuites;
import junit.framework.TestSuite;
import org.apache.ignite.internal.processors.cache.CacheNearReaderUpdateTest;
+import org.apache.ignite.internal.processors.cache.CacheRebalancingSelfTest;
import org.apache.ignite.internal.processors.cache.CacheSerializableTransactionsTest;
-import org.apache.ignite.internal.processors.cache.GridCacheSwapSpaceSpiConsistencySelfTest;
import org.apache.ignite.internal.processors.cache.EntryVersionConsistencyReadThroughTest;
+import org.apache.ignite.internal.processors.cache.GridCacheOffHeapCleanupTest;
+import org.apache.ignite.internal.processors.cache.GridCacheSwapSpaceSpiConsistencySelfTest;
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.GridCacheOffHeapCleanupTest;
import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentFairAffinityTest;
import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentNodeJoinValidationTest;
import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentTest;
-import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheSyncRebalanceModeSelfTest;
import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheTxIteratorSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheSyncRebalanceModeSelfTest;
import org.apache.ignite.internal.processors.cache.store.IgniteCacheWriteBehindNoUpdateSelfTest;
/**
@@ -62,6 +63,8 @@ public class IgniteCacheTestSuite5 extends TestSuite {
suite.addTestSuite(GridCacheOffHeapCleanupTest.class);
+ suite.addTestSuite(CacheRebalancingSelfTest.class);
+
return suite;
}
}
[07/25] ignite git commit: Merge remote-tracking branch
'remotes/professional/ignite-1.7.4'
Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/professional/ignite-1.7.4'
# Conflicts:
# modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/858b129e
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/858b129e
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/858b129e
Branch: refs/heads/ignite-4371
Commit: 858b129ec53f500130b57824a787a4af7c4eb6e7
Parents: 5099f14 bbaa79a
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Dec 8 12:44:49 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Dec 8 12:44:49 2016 +0300
----------------------------------------------------------------------
.../rendezvous/RendezvousAffinityFunction.java | 80 ++++--
.../discovery/GridDiscoveryManager.java | 118 +-------
.../GridCachePartitionExchangeManager.java | 134 +++------
.../processors/cache/GridCachePreloader.java | 11 +-
.../cache/GridCachePreloaderAdapter.java | 5 +-
.../processors/cache/GridCacheUtils.java | 17 --
.../binary/CacheObjectBinaryProcessorImpl.java | 3 +-
.../dht/GridClientPartitionTopology.java | 120 ++++----
.../dht/GridDhtPartitionTopology.java | 28 +-
.../dht/GridDhtPartitionTopologyImpl.java | 284 +++++++++++--------
.../dht/preloader/GridDhtPartitionDemander.java | 230 +++++++++------
.../dht/preloader/GridDhtPartitionFullMap.java | 18 +-
.../GridDhtPartitionsExchangeFuture.java | 56 +++-
.../dht/preloader/GridDhtPreloader.java | 9 +-
.../cache/transactions/IgniteTxHandler.java | 2 +-
.../service/GridServiceProcessor.java | 4 +-
.../internal/visor/query/VisorQueryJob.java | 2 +-
.../ignite/spi/discovery/tcp/ServerImpl.java | 41 ++-
.../tcp/internal/TcpDiscoveryStatistics.java | 4 +
.../AbstractAffinityFunctionSelfTest.java | 2 +-
.../GridDiscoveryManagerAliveCacheSelfTest.java | 2 +-
.../GridCachePartitionedAffinitySpreadTest.java | 7 +-
.../distributed/dht/GridCacheDhtTestUtils.java | 232 ---------------
.../GridCacheRebalancingSyncSelfTest.java | 2 +
.../h2/twostep/GridReduceQueryExecutor.java | 14 +-
25 files changed, 620 insertions(+), 805 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/858b129e/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/858b129e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/858b129e/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
index ee5b65c,43017db..8f8d78a
--- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
@@@ -104,24 -104,9 +104,25 @@@ public abstract class AbstractAffinityF
}
/**
+ * @param backups Number of backups.
* @throws Exception If failed.
*/
+ public void testNullKeyForPartitionCalculation() throws Exception {
+ AffinityFunction aff = affinityFunction();
+
+ try {
+ aff.partition(null);
+
+ fail("Should throw IllegalArgumentException due to NULL affinity key.");
+ } catch (IllegalArgumentException e) {
+ e.getMessage().contains("Null key is passed for a partition calculation. " +
+ "Make sure that an affinity key that is used is initialized properly.");
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
protected void checkNodeRemoved(int backups) throws Exception {
checkNodeRemoved(backups, 1, 1);
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/858b129e/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
[17/25] ignite git commit: IGNITE-3220 I/O bottleneck on
server/client cluster configuration Communications optimizations: -
possibility to open separate in/out connections - possibility to have
multiple connections between nodes - implemented NIO sessio
Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 1fe437c..b392c07 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -46,6 +46,7 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
import javax.net.ssl.SSLEngine;
import javax.net.ssl.SSLException;
import org.apache.ignite.Ignite;
@@ -53,6 +54,7 @@ import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteClientDisconnectedException;
import org.apache.ignite.IgniteException;
import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
import org.apache.ignite.cluster.ClusterNode;
import org.apache.ignite.configuration.AddressResolver;
import org.apache.ignite.configuration.IgniteConfiguration;
@@ -103,6 +105,7 @@ import org.apache.ignite.lang.IgniteBiTuple;
import org.apache.ignite.lang.IgniteFuture;
import org.apache.ignite.lang.IgniteInClosure;
import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.lang.IgniteProductVersion;
import org.apache.ignite.lang.IgniteRunnable;
import org.apache.ignite.lang.IgniteUuid;
import org.apache.ignite.plugin.extensions.communication.Message;
@@ -179,6 +182,7 @@ import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.SSL_META
* <li>Node local IP address (see {@link #setLocalAddress(String)})</li>
* <li>Node local port number (see {@link #setLocalPort(int)})</li>
* <li>Local port range (see {@link #setLocalPortRange(int)}</li>
+ * <li>Connections per node (see {@link #setConnectionsPerNode(int)})</li>
* <li>Connection buffer flush frequency (see {@link #setConnectionBufferFlushFrequency(long)})</li>
* <li>Connection buffer size (see {@link #setConnectionBufferSize(int)})</li>
* <li>Idle connection timeout (see {@link #setIdleConnectionTimeout(long)})</li>
@@ -238,6 +242,9 @@ import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.SSL_META
@IgniteSpiConsistencyChecked(optional = false)
public class TcpCommunicationSpi extends IgniteSpiAdapter
implements CommunicationSpi<Message>, TcpCommunicationSpiMBean {
+ /** */
+ private static final IgniteProductVersion MULTIPLE_CONN_SINCE_VER = IgniteProductVersion.fromString("1.8.0");
+
/** IPC error message. */
public static final String OUT_OF_RESOURCES_TCP_MSG = "Failed to allocate shared memory segment " +
"(switching to TCP, may be slower).";
@@ -257,11 +264,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
/** Node attribute that is mapped to node's external addresses (value is <tt>comm.tcp.ext-addrs</tt>). */
public static final String ATTR_EXT_ADDRS = "comm.tcp.ext-addrs";
+ /** */
+ public static final String ATTR_PAIRED_CONN = "comm.tcp.pairedConnection";
+
/** Default port which node sets listener to (value is <tt>47100</tt>). */
public static final int DFLT_PORT = 47100;
/** Default port which node sets listener for shared memory connections (value is <tt>48100</tt>). */
- public static final int DFLT_SHMEM_PORT = 48100;
+ public static final int DFLT_SHMEM_PORT = -1;
/** Default idle connection timeout (value is <tt>30000</tt>ms). */
public static final long DFLT_IDLE_CONN_TIMEOUT = 30000;
@@ -283,12 +293,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
/**
* Default count of selectors for TCP server equals to
- * {@code "Math.min(4, Runtime.getRuntime().availableProcessors())"}.
+ * {@code "Math.min(8, Runtime.getRuntime().availableProcessors())"}.
*/
- public static final int DFLT_SELECTORS_CNT = Math.min(4, Runtime.getRuntime().availableProcessors());
+ public static final int DFLT_SELECTORS_CNT = Math.max(4, Runtime.getRuntime().availableProcessors() / 2);
- /** Node ID meta for session. */
- private static final int NODE_ID_META = GridNioSessionMetaKey.nextUniqueKey();
+ /** Connection index meta for session. */
+ private static final int CONN_IDX_META = GridNioSessionMetaKey.nextUniqueKey();
/** Message tracker meta for session. */
private static final int TRACKER_META = GridNioSessionMetaKey.nextUniqueKey();
@@ -303,11 +313,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
public static final boolean DFLT_TCP_NODELAY = true;
/** Default received messages threshold for sending ack. */
- public static final int DFLT_ACK_SND_THRESHOLD = 16;
+ public static final int DFLT_ACK_SND_THRESHOLD = 32;
/** Default socket write timeout. */
public static final long DFLT_SOCK_WRITE_TIMEOUT = 2000;
+ /** Default connections per node. */
+ public static final int DFLT_CONN_PER_NODE = 1;
+
/** No-op runnable. */
private static final IgniteRunnable NOOP = new IgniteRunnable() {
@Override public void run() {
@@ -327,11 +340,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
/** */
private ConnectGateway connectGate;
+ /** */
+ private ConnectionPolicy connPlc;
+
/** Server listener. */
private final GridNioServerListener<Message> srvLsnr =
new GridNioServerListenerAdapter<Message>() {
@Override public void onSessionWriteTimeout(GridNioSession ses) {
- LT.warn(log, "Communication SPI Session write timed out (consider increasing " +
+ LT.warn(log,"Communication SPI session write timed out (consider increasing " +
"'socketWriteTimeout' " + "configuration property) [remoteAddr=" + ses.remoteAddress() +
", writeTimeout=" + sockWriteTimeout + ']');
@@ -347,46 +363,53 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
if (log.isDebugEnabled())
log.debug("Sending local node ID to newly accepted session: " + ses);
- ses.send(nodeIdMessage());
+ try {
+ ses.sendNoFuture(nodeIdMessage());
+ }
+ catch (IgniteCheckedException e) {
+ U.error(log, "Failed to send message: " + e, e);
+ }
}
}
@Override public void onDisconnected(GridNioSession ses, @Nullable Exception e) {
- UUID id = ses.meta(NODE_ID_META);
+ ConnectionKey connId = ses.meta(CONN_IDX_META);
- if (id != null) {
- GridCommunicationClient client = clients.get(id);
+ if (connId != null) {
+ UUID id = connId.nodeId();
- if (client instanceof GridTcpNioCommunicationClient &&
- ((GridTcpNioCommunicationClient) client).session() == ses) {
- client.close();
+ GridCommunicationClient[] nodeClients = clients.get(id);
- clients.remove(id, client);
+ if (nodeClients != null) {
+ for (GridCommunicationClient client : nodeClients) {
+ if (client instanceof GridTcpNioCommunicationClient &&
+ ((GridTcpNioCommunicationClient)client).session() == ses) {
+ client.close();
+
+ removeNodeClient(id, client);
+ }
+ }
}
if (!stopping) {
- boolean reconnect = false;
-
- GridNioRecoveryDescriptor recoveryData = ses.recoveryDescriptor();
-
- if (recoveryData != null) {
- if (recoveryData.nodeAlive(getSpiContext().node(id))) {
- if (!recoveryData.messagesFutures().isEmpty()) {
- reconnect = true;
+ GridNioRecoveryDescriptor outDesc = ses.outRecoveryDescriptor();
+ if (outDesc != null) {
+ if (outDesc.nodeAlive(getSpiContext().node(id))) {
+ if (!outDesc.messagesRequests().isEmpty()) {
if (log.isDebugEnabled())
log.debug("Session was closed but there are unacknowledged messages, " +
- "will try to reconnect [rmtNode=" + recoveryData.node().id() + ']');
+ "will try to reconnect [rmtNode=" + outDesc.node().id() + ']');
+
+ DisconnectedSessionInfo disconnectData =
+ new DisconnectedSessionInfo(outDesc, connId.connectionIndex());
+
+ commWorker.addProcessDisconnectRequest(disconnectData);
}
}
else
- recoveryData.onNodeLeft();
+ outDesc.onNodeLeft();
}
-
- DisconnectedSessionInfo disconnectData = new DisconnectedSessionInfo(recoveryData,
- reconnect);
-
- commWorker.addProcessDisconnectRequest(disconnectData);
}
CommunicationListener<Message> lsnr0 = lsnr;
@@ -403,21 +426,24 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
private void onFirstMessage(GridNioSession ses, Message msg) {
UUID sndId;
- if (msg instanceof NodeIdMessage)
- sndId = U.bytesToUuid(((NodeIdMessage)msg).nodeIdBytes, 0);
+ ConnectionKey connKey;
+
+ if (msg instanceof NodeIdMessage) {
+ sndId = U.bytesToUuid(((NodeIdMessage) msg).nodeIdBytes, 0);
+ connKey = new ConnectionKey(sndId, 0, -1);
+ }
else {
assert msg instanceof HandshakeMessage : msg;
+ HandshakeMessage msg0 = (HandshakeMessage)msg;
+
sndId = ((HandshakeMessage)msg).nodeId();
+ connKey = new ConnectionKey(sndId, msg0.connectionIndex(), msg0.connectCount());
}
if (log.isDebugEnabled())
log.debug("Remote node ID received: " + sndId);
- final UUID old = ses.addMeta(NODE_ID_META, sndId);
-
- assert old == null;
-
final ClusterNode rmtNode = getSpiContext().node(sndId);
if (rmtNode == null) {
@@ -429,57 +455,65 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
return;
}
+ final ConnectionKey old = ses.addMeta(CONN_IDX_META, connKey);
+
+ assert old == null;
+
ClusterNode locNode = getSpiContext().localNode();
if (ses.remoteAddress() == null)
return;
- GridCommunicationClient oldClient = clients.get(sndId);
+ assert msg instanceof HandshakeMessage : msg;
- boolean hasShmemClient = false;
+ HandshakeMessage msg0 = (HandshakeMessage)msg;
- if (oldClient != null) {
- if (oldClient instanceof GridTcpNioCommunicationClient) {
- if (log.isDebugEnabled())
- log.debug("Received incoming connection when already connected " +
- "to this node, rejecting [locNode=" + locNode.id() +
- ", rmtNode=" + sndId + ']');
+ if (usePairedConnections(rmtNode)) {
+ final GridNioRecoveryDescriptor recoveryDesc = inRecoveryDescriptor(rmtNode, connKey);
- ses.send(new RecoveryLastReceivedMessage(-1));
+ ConnectClosureNew c = new ConnectClosureNew(ses, recoveryDesc, rmtNode);
- return;
- }
+ boolean reserve = recoveryDesc.tryReserve(msg0.connectCount(), c);
+
+ if (reserve)
+ connectedNew(recoveryDesc, ses, true);
else {
- assert oldClient instanceof GridShmemCommunicationClient;
+ if (c.failed) {
+ ses.send(new RecoveryLastReceivedMessage(-1));
+
+ for (GridNioSession ses0 : nioSrvr.sessions()) {
+ ConnectionKey key0 = ses0.meta(CONN_IDX_META);
- hasShmemClient = true;
+ if (ses0.accepted() && key0 != null &&
+ key0.nodeId().equals(connKey.nodeId()) &&
+ key0.connectionIndex() == connKey.connectionIndex() &&
+ key0.connectCount() < connKey.connectCount())
+ ses0.close();
+ }
+ }
}
}
+ else {
+ assert connKey.connectionIndex() >= 0 : connKey;
- GridFutureAdapter<GridCommunicationClient> fut = new GridFutureAdapter<>();
-
- GridFutureAdapter<GridCommunicationClient> oldFut = clientFuts.putIfAbsent(sndId, fut);
-
- assert msg instanceof HandshakeMessage : msg;
-
- HandshakeMessage msg0 = (HandshakeMessage)msg;
+ GridCommunicationClient[] curClients = clients.get(sndId);
- final GridNioRecoveryDescriptor recoveryDesc = recoveryDescriptor(rmtNode);
+ GridCommunicationClient oldClient =
+ curClients != null && connKey.connectionIndex() < curClients.length ?
+ curClients[connKey.connectionIndex()] :
+ null;
- if (oldFut == null) {
- oldClient = clients.get(sndId);
+ boolean hasShmemClient = false;
if (oldClient != null) {
if (oldClient instanceof GridTcpNioCommunicationClient) {
if (log.isDebugEnabled())
log.debug("Received incoming connection when already connected " +
- "to this node, rejecting [locNode=" + locNode.id() +
- ", rmtNode=" + sndId + ']');
+ "to this node, rejecting [locNode=" + locNode.id() +
+ ", rmtNode=" + sndId + ']');
ses.send(new RecoveryLastReceivedMessage(-1));
- fut.onDone(oldClient);
-
return;
}
else {
@@ -489,51 +523,86 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
}
}
- boolean reserved = recoveryDesc.tryReserve(msg0.connectCount(),
- new ConnectClosure(ses, recoveryDesc, rmtNode, msg0, !hasShmemClient, fut));
+ GridFutureAdapter<GridCommunicationClient> fut = new GridFutureAdapter<>();
- if (log.isDebugEnabled())
- log.debug("Received incoming connection from remote node " +
+ GridFutureAdapter<GridCommunicationClient> oldFut = clientFuts.putIfAbsent(connKey, fut);
+
+ final GridNioRecoveryDescriptor recoveryDesc = inRecoveryDescriptor(rmtNode, connKey);
+
+ if (oldFut == null) {
+ curClients = clients.get(sndId);
+
+ oldClient = curClients != null && connKey.connectionIndex() < curClients.length ?
+ curClients[connKey.connectionIndex()] : null;
+
+ if (oldClient != null) {
+ if (oldClient instanceof GridTcpNioCommunicationClient) {
+ assert oldClient.connectionIndex() == connKey.connectionIndex() : oldClient;
+
+ if (log.isDebugEnabled())
+ log.debug("Received incoming connection when already connected " +
+ "to this node, rejecting [locNode=" + locNode.id() +
+ ", rmtNode=" + sndId + ']');
+
+ ses.send(new RecoveryLastReceivedMessage(-1));
+
+ fut.onDone(oldClient);
+
+ return;
+ }
+ else {
+ assert oldClient instanceof GridShmemCommunicationClient;
+
+ hasShmemClient = true;
+ }
+ }
+
+ boolean reserved = recoveryDesc.tryReserve(msg0.connectCount(),
+ new ConnectClosure(ses, recoveryDesc, rmtNode, connKey, msg0, !hasShmemClient, fut));
+
+ if (log.isDebugEnabled())
+ log.debug("Received incoming connection from remote node " +
"[rmtNode=" + rmtNode.id() + ", reserved=" + reserved + ']');
- if (reserved) {
- try {
- GridTcpNioCommunicationClient client =
+ if (reserved) {
+ try {
+ GridTcpNioCommunicationClient client =
connected(recoveryDesc, ses, rmtNode, msg0.received(), true, !hasShmemClient);
- fut.onDone(client);
- }
- finally {
- clientFuts.remove(rmtNode.id(), fut);
+ fut.onDone(client);
+ }
+ finally {
+ clientFuts.remove(connKey, fut);
+ }
}
}
- }
- else {
- if (oldFut instanceof ConnectFuture && locNode.order() < rmtNode.order()) {
- if (log.isDebugEnabled()) {
- log.debug("Received incoming connection from remote node while " +
+ else {
+ if (oldFut instanceof ConnectFuture && locNode.order() < rmtNode.order()) {
+ if (log.isDebugEnabled()) {
+ log.debug("Received incoming connection from remote node while " +
"connecting to this node, rejecting [locNode=" + locNode.id() +
", locNodeOrder=" + locNode.order() + ", rmtNode=" + rmtNode.id() +
", rmtNodeOrder=" + rmtNode.order() + ']');
- }
+ }
- ses.send(new RecoveryLastReceivedMessage(-1));
- }
- else {
- // The code below causes a race condition between shmem and TCP (see IGNITE-1294)
- boolean reserved = recoveryDesc.tryReserve(msg0.connectCount(),
- new ConnectClosure(ses, recoveryDesc, rmtNode, msg0, !hasShmemClient, fut));
+ ses.send(new RecoveryLastReceivedMessage(-1));
+ }
+ else {
+ // The code below causes a race condition between shmem and TCP (see IGNITE-1294)
+ boolean reserved = recoveryDesc.tryReserve(msg0.connectCount(),
+ new ConnectClosure(ses, recoveryDesc, rmtNode, connKey, msg0, !hasShmemClient, fut));
- if (reserved)
- connected(recoveryDesc, ses, rmtNode, msg0.received(), true, !hasShmemClient);
+ if (reserved)
+ connected(recoveryDesc, ses, rmtNode, msg0.received(), true, !hasShmemClient);
+ }
}
}
}
@Override public void onMessage(GridNioSession ses, Message msg) {
- UUID sndId = ses.meta(NODE_ID_META);
+ ConnectionKey connKey = ses.meta(CONN_IDX_META);
- if (sndId == null) {
+ if (connKey == null) {
assert ses.accepted() : ses;
if (!connectGate.tryEnter()) {
@@ -555,29 +624,37 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
else {
rcvdMsgsCnt.increment();
- GridNioRecoveryDescriptor recovery = ses.recoveryDescriptor();
+ if (msg instanceof RecoveryLastReceivedMessage) {
+ GridNioRecoveryDescriptor recovery = ses.outRecoveryDescriptor();
- if (recovery != null) {
- if (msg instanceof RecoveryLastReceivedMessage) {
+ if (recovery != null) {
RecoveryLastReceivedMessage msg0 = (RecoveryLastReceivedMessage)msg;
- if (log.isDebugEnabled())
- log.debug("Received recovery acknowledgement [rmtNode=" + sndId +
+ if (log.isDebugEnabled()) {
+ log.debug("Received recovery acknowledgement [rmtNode=" + connKey.nodeId() +
+ ", connIdx=" + connKey.connectionIndex() +
", rcvCnt=" + msg0.received() + ']');
+ }
recovery.ackReceived(msg0.received());
return;
}
- else {
+ }
+ else {
+ GridNioRecoveryDescriptor recovery = ses.inRecoveryDescriptor();
+
+ if (recovery != null) {
long rcvCnt = recovery.onReceived();
if (rcvCnt % ackSndThreshold == 0) {
- if (log.isDebugEnabled())
- log.debug("Send recovery acknowledgement [rmtNode=" + sndId +
+ if (log.isDebugEnabled()) {
+ log.debug("Send recovery acknowledgement [rmtNode=" + connKey.nodeId() +
+ ", connIdx=" + connKey.connectionIndex() +
", rcvCnt=" + rcvCnt + ']');
+ }
- nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(rcvCnt));
+ ses.systemMessage(new RecoveryLastReceivedMessage(rcvCnt));
recovery.lastAcknowledged(rcvCnt);
}
@@ -603,7 +680,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
else
c = NOOP;
- notifyListener(sndId, msg, c);
+ notifyListener(connKey.nodeId(), msg, c);
}
}
@@ -611,7 +688,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
* @param recovery Recovery descriptor.
* @param ses Session.
* @param node Node.
- * @param rcvCnt Number of received messages..
+ * @param rcvCnt Number of received messages.
* @param sndRes If {@code true} sends response for recovery handshake.
* @param createClient If {@code true} creates NIO communication client.
* @return Client.
@@ -623,32 +700,128 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
long rcvCnt,
boolean sndRes,
boolean createClient) {
+ ConnectionKey connKey = ses.meta(CONN_IDX_META);
+
+ assert connKey != null && connKey.connectionIndex() >= 0 : connKey;
+ assert !usePairedConnections(node);
+
recovery.onHandshake(rcvCnt);
- ses.recoveryDescriptor(recovery);
+ ses.inRecoveryDescriptor(recovery);
+ ses.outRecoveryDescriptor(recovery);
nioSrvr.resend(ses);
- if (sndRes)
- nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(recovery.received()));
+ try {
+ if (sndRes)
+ nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(recovery.received()));
+ }
+ catch (IgniteCheckedException e) {
+ U.error(log, "Failed to send message: " + e, e);
+ }
- recovery.connected();
+ recovery.onConnected();
GridTcpNioCommunicationClient client = null;
if (createClient) {
- client = new GridTcpNioCommunicationClient(ses, log);
-
- GridCommunicationClient oldClient = clients.putIfAbsent(node.id(), client);
+ client = new GridTcpNioCommunicationClient(connKey.connectionIndex(), ses, log);
- assert oldClient == null : "Client already created [node=" + node + ", client=" + client +
- ", oldClient=" + oldClient + ", recoveryDesc=" + recovery + ']';
+ addNodeClient(node, connKey.connectionIndex(), client);
}
return client;
}
/**
+ * @param recovery Recovery descriptor.
+ * @param ses Session.
+ * @param sndRes If {@code true} sends response for recovery handshake.
+ */
+ private void connectedNew(
+ GridNioRecoveryDescriptor recovery,
+ GridNioSession ses,
+ boolean sndRes) {
+ try {
+ ses.inRecoveryDescriptor(recovery);
+
+ if (sndRes)
+ nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(recovery.received()));
+
+ recovery.onConnected();
+ }
+ catch (IgniteCheckedException e) {
+ U.error(log, "Failed to send message: " + e, e);
+ }
+ }
+
+ /**
+ *
+ */
+ class ConnectClosureNew implements IgniteInClosure<Boolean> {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ private final GridNioSession ses;
+
+ /** */
+ private final GridNioRecoveryDescriptor recoveryDesc;
+
+ /** */
+ private final ClusterNode rmtNode;
+
+ /** */
+ private boolean failed;
+
+ /**
+ * @param ses Incoming session.
+ * @param recoveryDesc Recovery descriptor.
+ * @param rmtNode Remote node.
+ */
+ ConnectClosureNew(GridNioSession ses,
+ GridNioRecoveryDescriptor recoveryDesc,
+ ClusterNode rmtNode) {
+ this.ses = ses;
+ this.recoveryDesc = recoveryDesc;
+ this.rmtNode = rmtNode;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void apply(Boolean success) {
+ try {
+ failed = !success;
+
+ if (success) {
+ IgniteInClosure<IgniteInternalFuture<?>> lsnr = new IgniteInClosure<IgniteInternalFuture<?>>() {
+ @Override public void apply(IgniteInternalFuture<?> msgFut) {
+ try {
+ msgFut.get();
+
+ connectedNew(recoveryDesc, ses, false);
+ }
+ catch (IgniteCheckedException e) {
+ if (log.isDebugEnabled())
+ log.debug("Failed to send recovery handshake " +
+ "[rmtNode=" + rmtNode.id() + ", err=" + e + ']');
+
+ recoveryDesc.release();
+ }
+ }
+ };
+
+ nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(recoveryDesc.received()), lsnr);
+ }
+ else
+ nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(-1));
+ }
+ catch (IgniteCheckedException e) {
+ U.error(log, "Failed to send message: " + e, e);
+ }
+ }
+ }
+
+ /**
*
*/
@SuppressWarnings("PackageVisibleInnerClass")
@@ -674,10 +847,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
/** */
private final boolean createClient;
+ /** */
+ private final ConnectionKey connKey;
+
/**
* @param ses Incoming session.
* @param recoveryDesc Recovery descriptor.
* @param rmtNode Remote node.
+ * @param connKey Connection key.
* @param msg Handshake message.
* @param createClient If {@code true} creates NIO communication client..
* @param fut Connect future.
@@ -685,12 +862,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
ConnectClosure(GridNioSession ses,
GridNioRecoveryDescriptor recoveryDesc,
ClusterNode rmtNode,
+ ConnectionKey connKey,
HandshakeMessage msg,
boolean createClient,
GridFutureAdapter<GridCommunicationClient> fut) {
this.ses = ses;
this.recoveryDesc = recoveryDesc;
this.rmtNode = rmtNode;
+ this.connKey = connKey;
this.msg = msg;
this.createClient = createClient;
this.fut = fut;
@@ -699,39 +878,44 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
/** {@inheritDoc} */
@Override public void apply(Boolean success) {
if (success) {
- IgniteInClosure<IgniteInternalFuture<?>> lsnr = new IgniteInClosure<IgniteInternalFuture<?>>() {
- @Override public void apply(IgniteInternalFuture<?> msgFut) {
- try {
- msgFut.get();
+ try {
+ IgniteInClosure<IgniteInternalFuture<?>> lsnr = new IgniteInClosure<IgniteInternalFuture<?>>() {
+ @Override public void apply(IgniteInternalFuture<?> msgFut) {
+ try {
+ msgFut.get();
- GridTcpNioCommunicationClient client =
- connected(recoveryDesc, ses, rmtNode, msg.received(), false, createClient);
+ GridTcpNioCommunicationClient client =
+ connected(recoveryDesc, ses, rmtNode, msg.received(), false, createClient);
- fut.onDone(client);
- }
- catch (IgniteCheckedException e) {
- if (log.isDebugEnabled())
- log.debug("Failed to send recovery handshake " +
- "[rmtNode=" + rmtNode.id() + ", err=" + e + ']');
+ fut.onDone(client);
+ }
+ catch (IgniteCheckedException e) {
+ if (log.isDebugEnabled())
+ log.debug("Failed to send recovery handshake " +
+ "[rmtNode=" + rmtNode.id() + ", err=" + e + ']');
- recoveryDesc.release();
+ recoveryDesc.release();
- fut.onDone();
- }
- finally {
- clientFuts.remove(rmtNode.id(), fut);
+ fut.onDone();
+ }
+ finally {
+ clientFuts.remove(connKey, fut);
+ }
}
- }
- };
+ };
- nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(recoveryDesc.received()), lsnr);
+ nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(recoveryDesc.received()), lsnr);
+ }
+ catch (IgniteCheckedException e) {
+ U.error(log, "Failed to send message: " + e, e);
+ }
}
else {
try {
fut.onDone();
}
finally {
- clientFuts.remove(rmtNode.id(), fut);
+ clientFuts.remove(connKey, fut);
}
}
}
@@ -794,6 +978,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
/** Shared memory server. */
private IpcSharedMemoryServerEndpoint shmemSrv;
+ /** */
+ private boolean usePairedConnections = true;
+
+ /** */
+ private int connectionsPerNode = DFLT_CONN_PER_NODE;
+
/** {@code TCP_NODELAY} option value for created sockets. */
private boolean tcpNoDelay = DFLT_TCP_NODELAY;
@@ -816,7 +1006,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
private final Collection<ShmemWorker> shmemWorkers = new ConcurrentLinkedDeque8<>();
/** Clients. */
- private final ConcurrentMap<UUID, GridCommunicationClient> clients = GridConcurrentFactory.newMap();
+ private final ConcurrentMap<UUID, GridCommunicationClient[]> clients = GridConcurrentFactory.newMap();
/** SPI listener. */
private volatile CommunicationListener<Message> lsnr;
@@ -830,6 +1020,13 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
/** Count of selectors to use in TCP server. */
private int selectorsCnt = DFLT_SELECTORS_CNT;
+ /**
+ * Defines how many non-blocking {@code selector.selectNow()} should be made before
+ * falling into {@code selector.select(long)} in NIO server. Long value. Default is {@code 0}.
+ * Can be set to {@code Long.MAX_VALUE} so selector threads will never block.
+ */
+ private long selectorSpins = IgniteSystemProperties.getLong("IGNITE_SELECTOR_SPINS", 0L);
+
/** Address resolver. */
private AddressResolver addrRslvr;
@@ -863,11 +1060,17 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
};
/** Client connect futures. */
- private final ConcurrentMap<UUID, GridFutureAdapter<GridCommunicationClient>> clientFuts =
+ private final ConcurrentMap<ConnectionKey, GridFutureAdapter<GridCommunicationClient>> clientFuts =
GridConcurrentFactory.newMap();
/** */
- private final ConcurrentMap<ClientKey, GridNioRecoveryDescriptor> recoveryDescs = GridConcurrentFactory.newMap();
+ private final ConcurrentMap<ConnectionKey, GridNioRecoveryDescriptor> recoveryDescs = GridConcurrentFactory.newMap();
+
+ /** */
+ private final ConcurrentMap<ConnectionKey, GridNioRecoveryDescriptor> outRecDescs = GridConcurrentFactory.newMap();
+
+ /** */
+ private final ConcurrentMap<ConnectionKey, GridNioRecoveryDescriptor> inRecDescs = GridConcurrentFactory.newMap();
/** Discovery listener. */
private final GridLocalEventListener discoLsnr = new GridLocalEventListener() {
@@ -976,6 +1179,49 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
return locPortRange;
}
+ /** {@inheritDoc} */
+ @Override public boolean isUsePairedConnections() {
+ return usePairedConnections;
+ }
+
+ /**
+ * Set this to {@code true} if {@code TcpCommunicationSpi} should
+ * maintain connection for outgoing and incoming messages separately.
+ * In this case total number of connections between local and each remote node
+ * is {@link #getConnectionsPerNode()} * 2.
+ * <p>
+ * Set this to {@code false} if each connection of {@link #getConnectionsPerNode()}
+ * should be used for outgoing and incoming messages. In this case total number
+ * of connections between local and each remote node is {@link #getConnectionsPerNode()}.
+ * In this case load NIO selectors load
+ * balancing of {@link GridNioServer} will be disabled.
+ * <p>
+ * Default is {@code true}.
+ *
+ * @param usePairedConnections {@code true} to use paired connections and {@code false} otherwise.
+ * @see #getConnectionsPerNode()
+ */
+ public void setUsePairedConnections(boolean usePairedConnections) {
+ this.usePairedConnections = usePairedConnections;
+ }
+
+ /**
+ * Sets number of connections to each remote node. if {@link #isUsePairedConnections()}
+ * is {@code true} then number of connections is doubled and half is used for incoming and
+ * half for outgoing messages.
+ *
+ * @param maxConnectionsPerNode Number of connections per node.
+ * @see #isUsePairedConnections()
+ */
+ public void setConnectionsPerNode(int maxConnectionsPerNode) {
+ this.connectionsPerNode = maxConnectionsPerNode;
+ }
+
+ /** {@inheritDoc} */
+ @Override public int getConnectionsPerNode() {
+ return connectionsPerNode;
+ }
+
/**
* Sets local port to accept shared memory connections.
* <p>
@@ -1222,6 +1468,22 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
return selectorsCnt;
}
+ /** {@inheritDoc} */
+ @Override public long getSelectorSpins() {
+ return selectorSpins;
+ }
+
+ /**
+ * Defines how many non-blocking {@code selector.selectNow()} should be made before
+ * falling into {@code selector.select(long)} in NIO server. Long value. Default is {@code 0}.
+ * Can be set to {@code Long.MAX_VALUE} so selector threads will never block.
+ *
+ * @param selectorSpins Selector thread busy-loop iterations.
+ */
+ public void setSelectorSpins(long selectorSpins) {
+ this.selectorSpins = selectorSpins;
+ }
+
/**
* Sets value for {@code TCP_NODELAY} socket option. Each
* socket will be opened using provided value.
@@ -1396,7 +1658,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
if (log != null) {
StringBuilder sb = new StringBuilder("Communication SPI recovery descriptors: ").append(U.nl());
- for (Map.Entry<ClientKey, GridNioRecoveryDescriptor> entry : recoveryDescs.entrySet()) {
+ for (Map.Entry<ConnectionKey, GridNioRecoveryDescriptor> entry : recoveryDescs.entrySet()) {
GridNioRecoveryDescriptor desc = entry.getValue();
sb.append(" [key=").append(entry.getKey())
@@ -1409,14 +1671,48 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
.append(']').append(U.nl());
}
- sb.append("Communication SPI clients: ").append(U.nl());
+ for (Map.Entry<ConnectionKey, GridNioRecoveryDescriptor> entry : outRecDescs.entrySet()) {
+ GridNioRecoveryDescriptor desc = entry.getValue();
+
+ sb.append(" [key=").append(entry.getKey())
+ .append(", msgsSent=").append(desc.sent())
+ .append(", msgsAckedByRmt=").append(desc.acked())
+ .append(", reserveCnt=").append(desc.reserveCount())
+ .append(", connected=").append(desc.connected())
+ .append(", reserved=").append(desc.reserved())
+ .append(", descIdHash=").append(System.identityHashCode(desc))
+ .append(']').append(U.nl());
+ }
+
+ for (Map.Entry<ConnectionKey, GridNioRecoveryDescriptor> entry : inRecDescs.entrySet()) {
+ GridNioRecoveryDescriptor desc = entry.getValue();
- for (Map.Entry<UUID, GridCommunicationClient> entry : clients.entrySet()) {
- sb.append(" [node=").append(entry.getKey())
- .append(", client=").append(entry.getValue())
+ sb.append(" [key=").append(entry.getKey())
+ .append(", msgsRcvd=").append(desc.received())
+ .append(", lastAcked=").append(desc.lastAcknowledged())
+ .append(", reserveCnt=").append(desc.reserveCount())
+ .append(", connected=").append(desc.connected())
+ .append(", reserved=").append(desc.reserved())
+ .append(", handshakeIdx=").append(desc.handshakeIndex())
+ .append(", descIdHash=").append(System.identityHashCode(desc))
.append(']').append(U.nl());
}
+ sb.append("Communication SPI clients: ").append(U.nl());
+
+ for (Map.Entry<UUID, GridCommunicationClient[]> entry : clients.entrySet()) {
+ UUID nodeId = entry.getKey();
+ GridCommunicationClient[] clients0 = entry.getValue();
+
+ for (GridCommunicationClient client : clients0) {
+ if (client != null) {
+ sb.append(" [node=").append(nodeId)
+ .append(", client=").append(client)
+ .append(']').append(U.nl());
+ }
+ }
+ }
+
U.warn(log, sb.toString());
}
@@ -1426,6 +1722,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
nioSrvr.dumpStats();
}
+ /** */
+ private final ThreadLocal<Integer> threadConnIdx = new ThreadLocal<>();
+
+ /** */
+ private final AtomicInteger connIdx = new AtomicInteger();
+
/** {@inheritDoc} */
@Override public Map<String, Object> getNodeAttributes() throws IgniteSpiException {
initFailureDetectionTimeout();
@@ -1439,6 +1741,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
assertParameter(msgQueueLimit >= 0, "msgQueueLimit >= 0");
assertParameter(shmemPort > 0 || shmemPort == -1, "shmemPort > 0 || shmemPort == -1");
assertParameter(selectorsCnt > 0, "selectorsCnt > 0");
+ assertParameter(connectionsPerNode > 0, "connectionsPerNode > 0");
+ assertParameter(connectionsPerNode <= 1024, "connectionsPerNode <= 1024");
if (!failureDetectionTimeoutEnabled()) {
assertParameter(reconCnt > 0, "reconnectCnt > 0");
@@ -1458,6 +1762,21 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
"Specified 'unackedMsgsBufSize' is too low, it should be at least 'ackSndThreshold * 5'.");
}
+ if (connectionsPerNode > 1) {
+ connPlc = new ConnectionPolicy() {
+ @Override public int connectionIndex() {
+ return (int)(U.safeAbs(Thread.currentThread().getId()) % connectionsPerNode);
+ }
+ };
+ }
+ else {
+ connPlc = new ConnectionPolicy() {
+ @Override public int connectionIndex() {
+ return 0;
+ }
+ };
+ }
+
try {
locHost = U.resolveLocalHost(locAddr);
}
@@ -1495,6 +1814,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
res.put(createSpiAttributeName(ATTR_PORT), boundTcpPort);
res.put(createSpiAttributeName(ATTR_SHMEM_PORT), boundTcpShmemPort >= 0 ? boundTcpShmemPort : null);
res.put(createSpiAttributeName(ATTR_EXT_ADDRS), extAddrs);
+ res.put(createSpiAttributeName(ATTR_PAIRED_CONN), usePairedConnections);
return res;
}
@@ -1524,6 +1844,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
log.debug(configInfo("sockRcvBuf", sockRcvBuf));
log.debug(configInfo("shmemPort", shmemPort));
log.debug(configInfo("msgQueueLimit", msgQueueLimit));
+ log.debug(configInfo("connectionsPerNode", connectionsPerNode));
if (failureDetectionTimeoutEnabled()) {
log.debug(configInfo("connTimeout", connTimeout));
@@ -1548,6 +1869,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
", slowClientQueueLimit=" + slowClientQueueLimit + ']');
}
+ if (msgQueueLimit == 0)
+ U.quietAndWarn(log, "Message queue limit is set to 0 which may lead to " +
+ "potential OOMEs when running cache operations in FULL_ASYNC or PRIMARY_SYNC modes " +
+ "due to message queues growth on sender and reciever sides.");
+
registerMBean(gridName, this, TcpCommunicationSpiMBean.class);
connectGate = new ConnectGateway();
@@ -1642,9 +1968,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
assert formatter != null;
- UUID rmtNodeId = ses.meta(NODE_ID_META);
+ ConnectionKey key = ses.meta(CONN_IDX_META);
- return rmtNodeId != null ? formatter.reader(rmtNodeId, msgFactory) : null;
+ return key != null ? formatter.reader(key.nodeId(), msgFactory) : null;
}
};
@@ -1657,9 +1983,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
assert formatter != null;
- UUID rmtNodeId = ses.meta(NODE_ID_META);
+ ConnectionKey key = ses.meta(CONN_IDX_META);
- return rmtNodeId != null ? formatter.writer(rmtNodeId) : null;
+ return key != null ? formatter.writer(key.nodeId()) : null;
}
};
@@ -1716,6 +2042,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
.logger(log)
.selectorCount(selectorsCnt)
.gridName(gridName)
+ .serverName("tcp-comm")
.tcpNoDelay(tcpNoDelay)
.directBuffer(directBuf)
.byteOrder(ByteOrder.nativeOrder())
@@ -1725,18 +2052,21 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
.directMode(true)
.metricsListener(metricsLsnr)
.writeTimeout(sockWriteTimeout)
+ .selectorSpins(selectorSpins)
.filters(filters)
.writerFactory(writerFactory)
.skipRecoveryPredicate(skipRecoveryPred)
.messageQueueSizeListener(queueSizeMonitor)
+ .balancing(usePairedConnections) // Current balancing logic assumes separate in/out connections.
.build();
boundTcpPort = port;
// Ack Port the TCP server was bound to.
if (log.isInfoEnabled())
- log.info("Successfully bound to TCP port [port=" + boundTcpPort +
- ", locHost=" + locHost + ']');
+ log.info("Successfully bound communication NIO server to TCP port " +
+ "[port=" + boundTcpPort + ", locHost=" + locHost + ", selectorsCnt=" + selectorsCnt +
+ ", selectorSpins=" + srvr.selectorSpins() + ']');
srvr.idleTimeout(idleConnTimeout);
@@ -1837,8 +2167,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
shmemWorkers.clear();
// Force closing on stop (safety).
- for (GridCommunicationClient client : clients.values())
- client.forceClose();
+ for (GridCommunicationClient[] clients0 : clients.values()) {
+ for (GridCommunicationClient client : clients0) {
+ if (client != null)
+ client.forceClose();
+ }
+ }
// Clear resources.
nioSrvr = null;
@@ -1863,8 +2197,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
connectGate.stopped();
// Force closing.
- for (GridCommunicationClient client : clients.values())
- client.forceClose();
+ for (GridCommunicationClient[] clients0 : clients.values()) {
+ for (GridCommunicationClient client : clients0) {
+ if (client != null)
+ client.forceClose();
+ }
+ }
getSpiContext().deregisterPorts();
@@ -1875,8 +2213,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
@Override public void onClientDisconnected(IgniteFuture<?> reconnectFut) {
connectGate.disconnected(reconnectFut);
- for (GridCommunicationClient client : clients.values())
- client.forceClose();
+ for (GridCommunicationClient[] clients0 : clients.values()) {
+ for (GridCommunicationClient client : clients0) {
+ if (client != null)
+ client.forceClose();
+ }
+ }
IgniteClientDisconnectedCheckedException err = new IgniteClientDisconnectedCheckedException(reconnectFut,
"Failed to connect client node disconnected.");
@@ -1885,6 +2227,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
clientFut.onDone(err);
recoveryDescs.clear();
+ inRecDescs.clear();
+ outRecDescs.clear();
}
/** {@inheritDoc} */
@@ -1898,16 +2242,18 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
void onNodeLeft(UUID nodeId) {
assert nodeId != null;
- GridCommunicationClient client = clients.get(nodeId);
-
- if (client != null) {
- if (log.isDebugEnabled())
- log.debug("Forcing NIO client close since node has left [nodeId=" + nodeId +
- ", client=" + client + ']');
+ GridCommunicationClient[] clients0 = clients.remove(nodeId);
- client.forceClose();
+ if (clients0 != null) {
+ for (GridCommunicationClient client : clients0) {
+ if (client != null) {
+ if (log.isDebugEnabled())
+ log.debug("Forcing NIO client close since node has left [nodeId=" + nodeId +
+ ", client=" + client + ']');
- clients.remove(nodeId, client);
+ client.forceClose();
+ }
+ }
}
}
@@ -1982,11 +2328,13 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
else {
GridCommunicationClient client = null;
+ int connIdx = useMultipleConnections(node) ? connPlc.connectionIndex() : 0;
+
try {
boolean retry;
do {
- client = reserveClient(node);
+ client = reserveClient(node, connIdx);
UUID nodeId = null;
@@ -2000,7 +2348,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
if (!retry)
sentMsgsCnt.increment();
else {
- clients.remove(node.id(), client);
+ removeNodeClient(node.id(), client);
ClusterNode node0 = getSpiContext().node(node.id());
@@ -2017,26 +2365,94 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
throw new IgniteSpiException("Failed to send message to remote node: " + node, e);
}
finally {
- if (client != null && clients.remove(node.id(), client))
+ if (client != null && removeNodeClient(node.id(), client))
client.forceClose();
}
}
}
/**
+ * @param nodeId Node ID.
+ * @param rmvClient Client to remove.
+ * @return {@code True} if client was removed.
+ */
+ private boolean removeNodeClient(UUID nodeId, GridCommunicationClient rmvClient) {
+ for (;;) {
+ GridCommunicationClient[] curClients = clients.get(nodeId);
+
+ if (curClients == null || rmvClient.connectionIndex() >= curClients.length || curClients[rmvClient.connectionIndex()] != rmvClient)
+ return false;
+
+ GridCommunicationClient[] newClients = Arrays.copyOf(curClients, curClients.length);
+
+ newClients[rmvClient.connectionIndex()] = null;
+
+ if (clients.replace(nodeId, curClients, newClients))
+ return true;
+ }
+ }
+
+ /**
+ * @param node Node.
+ * @param connIdx Connection index.
+ * @param addClient Client to add.
+ */
+ private void addNodeClient(ClusterNode node, int connIdx, GridCommunicationClient addClient) {
+ assert connectionsPerNode > 0 : connectionsPerNode;
+ assert connIdx == addClient.connectionIndex() : addClient;
+
+ if (connIdx >= connectionsPerNode) {
+ assert !usePairedConnections(node);
+
+ return;
+ }
+
+ for (;;) {
+ GridCommunicationClient[] curClients = clients.get(node.id());
+
+ assert curClients == null || curClients[connIdx] == null : "Client already created [node=" + node.id() +
+ ", connIdx=" + connIdx +
+ ", client=" + addClient +
+ ", oldClient=" + curClients[connIdx] + ']';
+
+ GridCommunicationClient[] newClients;
+
+ if (curClients == null) {
+ newClients = new GridCommunicationClient[useMultipleConnections(node) ? connectionsPerNode : 1];
+ newClients[connIdx] = addClient;
+
+ if (clients.putIfAbsent(node.id(), newClients) == null)
+ break;
+ }
+ else {
+ newClients = Arrays.copyOf(curClients, curClients.length);
+ newClients[connIdx] = addClient;
+
+ if (clients.replace(node.id(), curClients, newClients))
+ break;
+ }
+ }
+ }
+
+ /**
* Returns existing or just created client to node.
*
* @param node Node to which client should be open.
+ * @param connIdx Connection index.
* @return The existing or just created client.
* @throws IgniteCheckedException Thrown if any exception occurs.
*/
- private GridCommunicationClient reserveClient(ClusterNode node) throws IgniteCheckedException {
+ private GridCommunicationClient reserveClient(ClusterNode node, int connIdx) throws IgniteCheckedException {
assert node != null;
+ assert (connIdx >= 0 && connIdx < connectionsPerNode) || !usePairedConnections(node) : connIdx;
UUID nodeId = node.id();
while (true) {
- GridCommunicationClient client = clients.get(nodeId);
+ GridCommunicationClient[] curClients = clients.get(nodeId);
+
+ GridCommunicationClient client = curClients != null && connIdx < curClients.length ?
+ curClients[connIdx] : null;
if (client == null) {
if (stopping)
@@ -2045,25 +2461,27 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
// Do not allow concurrent connects.
GridFutureAdapter<GridCommunicationClient> fut = new ConnectFuture();
- GridFutureAdapter<GridCommunicationClient> oldFut = clientFuts.putIfAbsent(nodeId, fut);
+ ConnectionKey connKey = new ConnectionKey(nodeId, connIdx, -1);
+
+ GridFutureAdapter<GridCommunicationClient> oldFut = clientFuts.putIfAbsent(connKey, fut);
if (oldFut == null) {
try {
- GridCommunicationClient client0 = clients.get(nodeId);
+ GridCommunicationClient[] curClients0 = clients.get(nodeId);
+
+ GridCommunicationClient client0 = curClients0 != null && connIdx < curClients0.length ?
+ curClients0[connIdx] : null;
if (client0 == null) {
- client0 = createNioClient(node);
+ client0 = createNioClient(node, connIdx);
if (client0 != null) {
- GridCommunicationClient old = clients.put(nodeId, client0);
-
- assert old == null : "Client already created " +
- "[node=" + node + ", client=" + client0 + ", oldClient=" + old + ']';
+ addNodeClient(node, connIdx, client0);
if (client0 instanceof GridTcpNioCommunicationClient) {
GridTcpNioCommunicationClient tcpClient = ((GridTcpNioCommunicationClient)client0);
- if (tcpClient.session().closeTime() > 0 && clients.remove(nodeId, client0)) {
+ if (tcpClient.session().closeTime() > 0 && removeNodeClient(nodeId, client0)) {
if (log.isDebugEnabled())
log.debug("Session was closed after client creation, will retry " +
"[node=" + node + ", client=" + client0 + ']');
@@ -2085,7 +2503,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
throw (Error)e;
}
finally {
- clientFuts.remove(nodeId, fut);
+ clientFuts.remove(connKey, fut);
}
}
else
@@ -2097,27 +2515,31 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
continue;
if (getSpiContext().node(nodeId) == null) {
- if (clients.remove(nodeId, client))
+ if (removeNodeClient(nodeId, client))
client.forceClose();
throw new IgniteSpiException("Destination node is not in topology: " + node.id());
}
}
+ assert connIdx == client.connectionIndex() : client;
+
if (client.reserve())
return client;
else
// Client has just been closed by idle worker. Help it and try again.
- clients.remove(nodeId, client);
+ removeNodeClient(nodeId, client);
}
}
/**
* @param node Node to create client for.
+ * @param connIdx Connection index.
* @return Client.
* @throws IgniteCheckedException If failed.
*/
- @Nullable protected GridCommunicationClient createNioClient(ClusterNode node) throws IgniteCheckedException {
+ @Nullable private GridCommunicationClient createNioClient(ClusterNode node, int connIdx)
+ throws IgniteCheckedException {
assert node != null;
Integer shmemPort = node.attribute(createSpiAttributeName(ATTR_SHMEM_PORT));
@@ -2136,6 +2558,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
try {
GridCommunicationClient client = createShmemClient(
node,
+ connIdx,
shmemPort);
if (log.isDebugEnabled())
@@ -2158,7 +2581,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
connectGate.enter();
try {
- GridCommunicationClient client = createTcpClient(node);
+ GridCommunicationClient client = createTcpClient(node, connIdx);
if (log.isDebugEnabled())
log.debug("TCP client created: " + client);
@@ -2173,10 +2596,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
/**
* @param node Node.
* @param port Port.
+ * @param connIdx Connection index.
* @return Client.
* @throws IgniteCheckedException If failed.
*/
- @Nullable protected GridCommunicationClient createShmemClient(ClusterNode node,
+ @Nullable private GridCommunicationClient createShmemClient(ClusterNode node,
+ int connIdx,
Integer port) throws IgniteCheckedException {
int attempt = 1;
@@ -2190,7 +2615,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
GridCommunicationClient client;
try {
- client = new GridShmemCommunicationClient(metricsLsnr,
+ client = new GridShmemCommunicationClient(
+ connIdx,
+ metricsLsnr,
port,
timeoutHelper.nextTimeoutChunk(connTimeout),
log,
@@ -2211,7 +2638,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
}
try {
- safeHandshake(client, null, node.id(), timeoutHelper.nextTimeoutChunk(connTimeout0), null);
+ safeHandshake(client,
+ null,
+ node.id(),
+ timeoutHelper.nextTimeoutChunk(connTimeout0),
+ null,
+ null);
}
catch (HandshakeTimeoutException | IgniteSpiOperationTimeoutException e) {
client.forceClose();
@@ -2270,10 +2702,10 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
*/
private void checkClientQueueSize(GridNioSession ses, int msgQueueSize) {
if (slowClientQueueLimit > 0 && msgQueueSize > slowClientQueueLimit) {
- UUID id = ses.meta(NODE_ID_META);
+ ConnectionKey id = ses.meta(CONN_IDX_META);
if (id != null) {
- ClusterNode node = getSpiContext().node(id);
+ ClusterNode node = getSpiContext().node(id.nodeId);
if (node != null && node.isClient()) {
String msg = "Client node outbound message queue size exceeded slowClientQueueLimit, " +
@@ -2283,11 +2715,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
", clientNode=" + node +
", slowClientQueueLimit=" + slowClientQueueLimit + ']';
- U.quietAndWarn(
- log,
- msg);
+ U.quietAndWarn(log, msg);
- getSpiContext().failNode(id, msg);
+ getSpiContext().failNode(id.nodeId(), msg);
}
}
}
@@ -2297,10 +2727,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
* Establish TCP connection to remote node and returns client.
*
* @param node Remote node.
+ * @param connIdx Connection index.
* @return Client.
* @throws IgniteCheckedException If failed.
*/
- protected GridCommunicationClient createTcpClient(ClusterNode node) throws IgniteCheckedException {
+ protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) throws IgniteCheckedException {
Collection<String> rmtAddrs0 = node.attribute(createSpiAttributeName(ATTR_ADDRS));
Collection<String> rmtHostNames0 = node.attribute(createSpiAttributeName(ATTR_HOST_NAMES));
Integer boundPort = node.attribute(createSpiAttributeName(ATTR_PORT));
@@ -2368,7 +2799,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
"(node left topology): " + node);
}
- GridNioRecoveryDescriptor recoveryDesc = recoveryDescriptor(node);
+ ConnectionKey connKey = new ConnectionKey(node.id(), connIdx, -1);
+
+ GridNioRecoveryDescriptor recoveryDesc = outRecoveryDescriptor(node, connKey);
if (!recoveryDesc.reserve()) {
U.closeQuiet(ch);
@@ -2395,11 +2828,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
sslMeta.sslEngine(sslEngine);
}
+ Integer handshakeConnIdx = useMultipleConnections(node) ? connIdx : null;
+
rcvCnt = safeHandshake(ch,
recoveryDesc,
node.id(),
timeoutHelper.nextTimeoutChunk(connTimeout0),
- sslMeta);
+ sslMeta,
+ handshakeConnIdx);
if (rcvCnt == -1)
return null;
@@ -2410,7 +2846,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
}
try {
- meta.put(NODE_ID_META, node.id());
+ meta.put(CONN_IDX_META, connKey);
if (recoveryDesc != null) {
recoveryDesc.onHandshake(rcvCnt);
@@ -2420,7 +2856,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
GridNioSession ses = nioSrvr.createSession(ch, meta).get();
- client = new GridTcpNioCommunicationClient(ses, log);
+ client = new GridTcpNioCommunicationClient(connIdx, ses, log);
conn = true;
}
@@ -2564,6 +3000,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
* @param rmtNodeId Remote node.
* @param timeout Timeout for handshake.
* @param sslMeta Session meta.
+ * @param handshakeConnIdx Non null connection index if need send it in handshake.
* @throws IgniteCheckedException If handshake failed or wasn't completed withing timeout.
* @return Handshake response.
*/
@@ -2573,7 +3010,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
@Nullable GridNioRecoveryDescriptor recovery,
UUID rmtNodeId,
long timeout,
- GridSslMeta sslMeta
+ GridSslMeta sslMeta,
+ @Nullable Integer handshakeConnIdx
) throws IgniteCheckedException {
HandshakeTimeoutObject<T> obj = new HandshakeTimeoutObject<>(client, U.currentTimeMillis() + timeout);
@@ -2655,14 +3093,28 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
"fully initialized [isStopping=" + getSpiContext().isStopping() + ']');
if (recovery != null) {
- HandshakeMessage msg = new HandshakeMessage(locNode.id(),
- recovery.incrementConnectCount(),
- recovery.received());
+ HandshakeMessage msg;
+
+ int msgSize = 33;
+
+ if (handshakeConnIdx != null) {
+ msg = new HandshakeMessage2(locNode.id(),
+ recovery.incrementConnectCount(),
+ recovery.received(),
+ handshakeConnIdx);
+
+ msgSize += 4;
+ }
+ else {
+ msg = new HandshakeMessage(locNode.id(),
+ recovery.incrementConnectCount(),
+ recovery.received());
+ }
if (log.isDebugEnabled())
log.debug("Write handshake message [rmtNode=" + rmtNodeId + ", msg=" + msg + ']');
- buf = ByteBuffer.allocate(33);
+ buf = ByteBuffer.allocate(msgSize);
buf.order(ByteOrder.nativeOrder());
@@ -2689,6 +3141,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
else
ch.write(ByteBuffer.wrap(nodeIdMessage().nodeIdBytesWithType));
}
+
if (recovery != null) {
if (log.isDebugEnabled())
log.debug("Waiting for handshake [rmtNode=" + rmtNodeId + ']');
@@ -2818,26 +3271,81 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
U.join(commWorker, log);
- for (GridCommunicationClient client : clients.values())
- client.forceClose();
+ for (GridCommunicationClient[] clients0 : clients.values()) {
+ for (GridCommunicationClient client : clients0) {
+ if (client != null)
+ client.forceClose();
+ }
+ }
+ }
+
+ /**
+ * @param node Node.
+ * @param key Connection key.
+ * @return Recovery descriptor for outgoing connection.
+ */
+ private GridNioRecoveryDescriptor outRecoveryDescriptor(ClusterNode node, ConnectionKey key) {
+ if (usePairedConnections(node))
+ return recoveryDescriptor(outRecDescs, true, node, key);
+ else
+ return recoveryDescriptor(recoveryDescs, false, node, key);
}
/**
* @param node Node.
- * @return Recovery receive data for given node.
+ * @param key Connection key.
+ * @return Recovery descriptor for incoming connection.
+ */
+ private GridNioRecoveryDescriptor inRecoveryDescriptor(ClusterNode node, ConnectionKey key) {
+ if (usePairedConnections(node))
+ return recoveryDescriptor(inRecDescs, true, node, key);
+ else
+ return recoveryDescriptor(recoveryDescs, false, node, key);
+ }
+
+ /**
+ * @param node Node.
+ * @return {@code True} if given node supports multiple connections per-node for communication.
+ */
+ private boolean useMultipleConnections(ClusterNode node) {
+ return node.version().compareToIgnoreTimestamp(MULTIPLE_CONN_SINCE_VER) >= 0;
+ }
+
+ /**
+ * @param node Node.
+ * @return {@code True} if can use in/out connection pair for communication.
*/
- private GridNioRecoveryDescriptor recoveryDescriptor(ClusterNode node) {
- ClientKey id = new ClientKey(node.id(), node.order());
+ private boolean usePairedConnections(ClusterNode node) {
+ if (usePairedConnections) {
+ Boolean attr = node.attribute(createSpiAttributeName(ATTR_PAIRED_CONN));
+
+ return attr != null && attr;
+ }
+
+ return false;
+ }
- GridNioRecoveryDescriptor recovery = recoveryDescs.get(id);
+ /**
+ * @param recoveryDescs Descriptors map.
+ * @param pairedConnections {@code True} if in/out connections pair is used for communication with node.
+ * @param node Node.
+ * @param key Connection key.
+ * @return Recovery receive data for given node.
+ */
+ private GridNioRecoveryDescriptor recoveryDescriptor(
+ ConcurrentMap<ConnectionKey, GridNioRecoveryDescriptor> recoveryDescs,
+ boolean pairedConnections,
+ ClusterNode node,
+ ConnectionKey key) {
+ GridNioRecoveryDescriptor recovery = recoveryDescs.get(key);
if (recovery == null) {
int maxSize = Math.max(msgQueueLimit, ackSndThreshold);
- int queueLimit = unackedMsgsBufSize != 0 ? unackedMsgsBufSize : (maxSize * 5);
+ int queueLimit = unackedMsgsBufSize != 0 ? unackedMsgsBufSize : (maxSize * 128);
- GridNioRecoveryDescriptor old =
- recoveryDescs.putIfAbsent(id, recovery = new GridNioRecoveryDescriptor(queueLimit, node, log));
+ GridNioRecoveryDescriptor old = recoveryDescs.putIfAbsent(key,
+ recovery = new GridNioRecoveryDescriptor(pairedConnections, queueLimit, node, log));
if (old != null)
recovery = old;
@@ -2879,54 +3387,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
return S.toString(TcpCommunicationSpi.class, this);
}
- /**
- *
- */
- private static class ClientKey {
- /** */
- private UUID nodeId;
-
- /** */
- private long order;
-
- /**
- * @param nodeId Node ID.
- * @param order Node order.
- */
- private ClientKey(UUID nodeId, long order) {
- this.nodeId = nodeId;
- this.order = order;
- }
-
- /** {@inheritDoc} */
- @Override public boolean equals(Object obj) {
- if (this == obj)
- return true;
-
- if (obj == null || getClass() != obj.getClass())
- return false;
-
- ClientKey other = (ClientKey)obj;
-
- return order == other.order && nodeId.equals(other.nodeId);
-
- }
-
- /** {@inheritDoc} */
- @Override public int hashCode() {
- int res = nodeId.hashCode();
-
- res = 31 * res + (int)(order ^ (order >>> 32));
-
- return res;
- }
-
- /** {@inheritDoc} */
- @Override public String toString() {
- return S.toString(ClientKey.class, this);
- }
- }
-
/** Internal exception class for proper timeout handling. */
private static class HandshakeTimeoutException extends IgniteCheckedException {
/** */
@@ -3026,9 +3486,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
assert formatter != null;
- UUID rmtNodeId = ses.meta(NODE_ID_META);
+ ConnectionKey connKey = ses.meta(CONN_IDX_META);
- return rmtNodeId != null ? formatter.writer(rmtNodeId) : null;
+ return connKey != null ? formatter.writer(connKey.nodeId()) : null;
}
};
@@ -3042,9 +3502,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
assert formatter != null;
- UUID rmtNodeId = ses.meta(NODE_ID_META);
+ ConnectionKey connKey = ses.meta(CONN_IDX_META);
- return rmtNodeId != null ? formatter.reader(rmtNodeId, msgFactory) : null;
+ return connKey != null ? formatter.reader(connKey.nodeId(), msgFactory) : null;
}
};
@@ -3125,62 +3585,108 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
private void processIdle() {
cleanupRecovery();
- for (Map.Entry<UUID, GridCommunicationClient> e : clients.entrySet()) {
+ for (Map.Entry<UUID, GridCommunicationClient[]> e : clients.entrySet()) {
UUID nodeId = e.getKey();
- GridCommunicationClient client = e.getValue();
+ for (GridCommunicationClient client : e.getValue()) {
+ if (client == null)
+ continue;
- ClusterNode node = getSpiContext().node(nodeId);
+ ClusterNode node = getSpiContext().node(nodeId);
- if (node == null) {
- if (log.isDebugEnabled())
- log.debug("Forcing close of non-existent node connection: " + nodeId);
+ if (node == null) {
+ if (log.isDebugEnabled())
+ log.debug("Forcing close of non-existent node connection: " + nodeId);
- client.forceClose();
+ client.forceClose();
- clients.remove(nodeId, client);
+ removeNodeClient(nodeId, client);
- continue;
- }
+ continue;
+ }
- GridNioRecoveryDescriptor recovery = null;
+ GridNioRecoveryDescriptor recovery = null;
- if (client instanceof GridTcpNioCommunicationClient) {
- recovery = recoveryDescs.get(new ClientKey(node.id(), node.order()));
+ if (!usePairedConnections(node) && client instanceof GridTcpNioCommunicationClient) {
+ recovery = recoveryDescs.get(new ConnectionKey(node.id(), client.connectionIndex(), -1));
- if (recovery != null && recovery.lastAcknowledged() != recovery.received()) {
- RecoveryLastReceivedMessage msg = new RecoveryLastReceivedMessage(recovery.received());
+ if (recovery != null && recovery.lastAcknowledged() != recovery.received()) {
+ RecoveryLastReceivedMessage msg = new RecoveryLastReceivedMessage(recovery.received());
- if (log.isDebugEnabled())
- log.debug("Send recovery acknowledgement on timeout [rmtNode=" + nodeId +
- ", rcvCnt=" + msg.received() + ']');
+ if (log.isDebugEnabled())
+ log.debug("Send recovery acknowledgement on timeout [rmtNode=" + nodeId +
+ ", rcvCnt=" + msg.received() + ']');
- nioSrvr.sendSystem(((GridTcpNioCommunicationClient)client).session(), msg);
+ try {
+ nioSrvr.sendSystem(((GridTcpNioCommunicationClient)client).session(), msg);
- recovery.lastAcknowledged(msg.received());
+ recovery.lastAcknowledged(msg.received());
+ }
+ catch (IgniteCheckedException err) {
+ U.error(log, "Failed to send message: " + err, err);
+ }
- continue;
+ continue;
+ }
}
- }
- long idleTime = client.getIdleTime();
+ long idleTime = client.getIdleTime();
+
+ if (idleTime >= idleConnTimeout) {
+ if (recovery == null && usePairedConnections(node))
+ recovery = outRecDescs.get(new ConnectionKey(node.id(), client.connectionIndex(), -1));
+
+ if (recovery != null &&
+ recovery.nodeAlive(getSpiContext().node(nodeId)) &&
+ !recovery.messagesRequests().isEmpty()) {
+ if (log.isDebugEnabled())
+ log.debug("Node connection is idle, but there are unacknowledged messages, " +
+ "will wait: " + nodeId);
+
+ continue;
+ }
- if (idleTime >= idleConnTimeout) {
- if (recovery != null &&
- recovery.nodeAlive(getSpiContext().node(nodeId)) &&
- !recovery.messagesFutures().isEmpty()) {
if (log.isDebugEnabled())
- log.debug("Node connection is idle, but there are unacknowledged messages, " +
- "will wait: " + nodeId);
+ log.debug("Closing idle node connection: " + nodeId);
- continue;
+ if (client.close() || client.closed())
+ removeNodeClient(nodeId, client);
}
+ }
+ }
- if (log.isDebugEnabled())
- log.debug("Closing idle node connection: " + nodeId);
+ for (GridNioSession ses : nioSrvr.sessions()) {
+ GridNioRecoveryDescriptor recovery = ses.inRecoveryDescriptor();
+
+ if (recovery != null && usePairedConnections(recovery.node())) {
+ assert ses.accepted() : ses;
+
+ sendAckOnTimeout(recovery, ses);
+ }
+ }
+ }
+
+ /**
+ * @param recovery Recovery descriptor.
+ * @param ses Session.
+ */
+ private void sendAckOnTimeout(GridNioRecoveryDescriptor recovery, GridNioSession ses) {
+ if (recovery != null && recovery.lastAcknowledged() != recovery.received()) {
+ RecoveryLastReceivedMessage msg = new RecoveryLastReceivedMessage(recovery.received());
+
+ if (log.isDebugEnabled()) {
+ log.debug("Send recovery acknowledgement on timeout [rmtNode=" + recovery.node().id() +
+ ", rcvCnt=" + msg.received() +
+ ", lastAcked=" + recovery.lastAcknowledged() + ']');
+ }
+
+ try {
+ nioSrvr.sendSystem(ses, msg);
- if (client.close() || client.closed())
- clients.remove(nodeId, client);
+ recovery.lastAcknowledged(msg.received());
+ }
+ catch (IgniteCheckedException e) {
+ U.error(log, "Failed to send message: " + e, e);
}
}
}
@@ -3189,15 +3695,24 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
*
*/
private void cleanupRecovery() {
- Set<ClientKey> left = null;
+ cleanupRecovery(recoveryDescs);
+ cleanupRecovery(inRecDescs);
+ cleanupRecovery(outRecDescs);
+ }
+
+ /**
+ * @param recoveryDescs Recovery descriptors to cleanup.
+ */
+ private void cleanupRecovery(ConcurrentMap<ConnectionKey, GridNioRecoveryDescriptor> recoveryDescs) {
+ Set<ConnectionKey> left = null;
- for (Map.Entry<ClientKey, GridNioRecoveryDescriptor> e : recoveryDescs.entrySet()) {
+ for (Map.Entry<ConnectionKey, GridNioRecoveryDescriptor> e : recoveryDescs.entrySet()) {
if (left != null && left.contains(e.getKey()))
continue;
- GridNioRecoveryDescriptor recoverySnd = e.getValue();
+ GridNioRecoveryDescriptor recoveryDesc = e.getValue();
- if (!recoverySnd.nodeAlive(getSpiContext().node(recoverySnd.node().id()))) {
+ if (!recoveryDesc.nodeAlive(getSpiContext().node(e.getKey().nodeId()))) {
if (left == null)
left = new HashSet<>();
@@ -3208,11 +3723,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
if (left != null) {
assert !left.isEmpty();
- for (ClientKey id : left) {
- GridNioRecoveryDescriptor recoverySnd = recoveryDescs.get(id);
+ for (ConnectionKey id : left) {
+ GridNioRecoveryDescriptor recoveryDesc = recoveryDescs.get(id);
- if (recoverySnd != null && recoverySnd.onNodeLeft())
- recoveryDescs.remove(id);
+ if (recoveryDesc != null && recoveryDesc.onNodeLeft())
+ recoveryDescs.remove(id, recoveryDesc);
}
}
}
@@ -3221,45 +3736,43 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
* @param sesInfo Disconnected session information.
*/
private void processDisconnect(DisconnectedSessionInfo sesInfo) {
- if (sesInfo.reconnect) {
- GridNioRecoveryDescriptor recoveryDesc = sesInfo.recoveryDesc;
-
- ClusterNode node = recoveryDesc.node();
+ GridNioRecoveryDescriptor recoveryDesc = sesInfo.recoveryDesc;
- if (!recoveryDesc.nodeAlive(getSpiContext().node(node.id())))
- return;
+ ClusterNode node = recoveryDesc.node();
- try {
- if (log.isDebugEnabled())
- log.debug("Recovery reconnect [rmtNode=" + recoveryDesc.node().id() + ']');
+ if (!recoveryDesc.nodeAlive(getSpiContext().node(node.id())))
+ return;
- GridCommunicationClient client = reserveClient(node);
+ try {
+ if (log.isDebugEnabled())
+ log.debug("Recovery reconnect [rmtNode=" + recoveryDesc.node().id() + ']');
- client.release();
- }
- catch (IgniteCheckedException | IgniteException e) {
- try {
- if (recoveryDesc.nodeAlive(getSpiContext().node(node.id())) && getSpiContext().pingNode(node.id())) {
- if (log.isDebugEnabled())
- log.debug("Recovery reconnect failed, will retry " +
- "[rmtNode=" + recoveryDesc.node().id() + ", err=" + e + ']');
+ GridCommunicationClient client = reserveClient(node, sesInfo.connIdx);
- addProcessDisconnectRequest(sesInfo);
- }
- else {
- if (log.isDebugEnabled())
- log.debug("Recovery reconnect failed, " +
- "node left [rmtNode=" + recoveryDesc.node().id() + ", err=" + e + ']');
+ client.release();
+ }
+ catch (IgniteCheckedException | IgniteException e) {
+ try {
+ if (recoveryDesc.nodeAlive(getSpiContext().node(node.id())) && getSpiContext().pingNode(node.id())) {
+ if (log.isDebugEnabled())
+ log.debug("Recovery reconnect failed, will retry " +
+ "[rmtNode=" + recoveryDesc.node().id() + ", err=" + e + ']');
- onException("Recovery reconnect failed, node left [rmtNode=" + recoveryDesc.node().id() + "]",
- e);
- }
+ addProcessDisconnectRequest(sesInfo);
}
- catch (IgniteClientDisconnectedException e0) {
+ else {
if (log.isDebugEnabled())
- log.debug("Failed to ping node, client disconnected.");
+ log.debug("Recovery reconnect failed, " +
+ "node left [rmtNode=" + recoveryDesc.node().id() + ", err=" + e + ']');
+
+ onE
<TRUNCATED>
[10/25] ignite git commit: IGNITE-4027 Extract PlatformTargetProxy
interface
Posted by sb...@apache.org.
IGNITE-4027 Extract PlatformTargetProxy interface
This closes #1188
Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/59e6fec0
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/59e6fec0
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/59e6fec0
Branch: refs/heads/ignite-4371
Commit: 59e6fec0b92c353ee5e128b9343a59f4b99bd468
Parents: 597f3a5
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Thu Dec 8 14:53:16 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Thu Dec 8 14:53:16 2016 +0300
----------------------------------------------------------------------
.../platform/PlatformAbstractTarget.java | 268 +++----------------
.../platform/PlatformAsyncTarget.java | 44 +++
.../platform/PlatformNoopProcessor.java | 41 +--
.../processors/platform/PlatformProcessor.java | 42 +--
.../platform/PlatformProcessorImpl.java | 87 +++---
.../processors/platform/PlatformTarget.java | 103 ++++---
.../platform/PlatformTargetProxy.java | 126 +++++++++
.../platform/PlatformTargetProxyImpl.java | 222 +++++++++++++++
.../binary/PlatformBinaryProcessor.java | 6 +-
.../platform/cache/PlatformCache.java | 15 +-
.../platform/cache/PlatformCacheIterator.java | 2 +-
.../cache/affinity/PlatformAffinity.java | 4 +-
.../affinity/PlatformAffinityFunction.java | 7 +-
.../PlatformAffinityFunctionTarget.java | 4 +-
.../query/PlatformAbstractQueryCursor.java | 4 +-
.../query/PlatformContinuousQueryProxy.java | 3 +-
.../callback/PlatformCallbackGateway.java | 6 +-
.../callback/PlatformCallbackUtils.java | 6 +-
.../platform/cluster/PlatformClusterGroup.java | 18 +-
.../platform/compute/PlatformCompute.java | 15 +-
.../datastreamer/PlatformDataStreamer.java | 4 +-
.../PlatformStreamReceiverImpl.java | 8 +-
.../datastructures/PlatformAtomicLong.java | 4 +-
.../datastructures/PlatformAtomicReference.java | 8 +-
.../datastructures/PlatformAtomicSequence.java | 2 +-
.../platform/events/PlatformEvents.java | 15 +-
.../platform/messaging/PlatformMessaging.java | 9 +-
.../platform/services/PlatformServices.java | 27 +-
.../transactions/PlatformTransactions.java | 8 +-
.../platform/utils/PlatformFutureUtils.java | 14 +-
.../utils/PlatformListenableTarget.java | 62 +++++
.../cpp/jni/include/ignite/jni/exports.h | 3 -
.../platforms/cpp/jni/include/ignite/jni/java.h | 7 -
modules/platforms/cpp/jni/project/vs/module.def | 2 -
modules/platforms/cpp/jni/src/exports.cpp | 8 -
modules/platforms/cpp/jni/src/java.cpp | 76 ++----
.../Apache.Ignite.Core.csproj | 1 +
.../Apache.Ignite.Core/Impl/Common/Future.cs | 13 +-
.../Impl/Common/Listenable.cs | 49 ++++
.../Impl/Compute/ComputeImpl.cs | 4 +-
.../Apache.Ignite.Core/Impl/PlatformTarget.cs | 2 +-
.../Impl/Unmanaged/IgniteJniNativeMethods.cs | 8 -
.../Impl/Unmanaged/UnmanagedUtils.cs | 5 -
43 files changed, 817 insertions(+), 545 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
index 2df86ac..506470b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
@@ -24,16 +24,16 @@ import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.binary.BinaryRawReaderEx;
import org.apache.ignite.internal.binary.BinaryRawWriterEx;
import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
-import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils;
import org.apache.ignite.internal.processors.platform.utils.PlatformListenable;
+import org.apache.ignite.internal.processors.platform.utils.PlatformListenableTarget;
import org.apache.ignite.lang.IgniteFuture;
import org.jetbrains.annotations.Nullable;
/**
* Abstract interop target.
*/
-public abstract class PlatformAbstractTarget implements PlatformTarget {
+public abstract class PlatformAbstractTarget implements PlatformTarget, PlatformAsyncTarget {
/** Constant: TRUE.*/
protected static final int TRUE = 1;
@@ -60,144 +60,6 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
log = platformCtx.kernalContext().log(PlatformAbstractTarget.class);
}
- /** {@inheritDoc} */
- @Override public long inLongOutLong(int type, long val) throws Exception {
- try {
- return processInLongOutLong(type, val);
- }
- catch (Exception e) {
- throw convertException(e);
- }
- }
-
- /** {@inheritDoc} */
- @Override public long inStreamOutLong(int type, long memPtr) throws Exception {
- try (PlatformMemory mem = platformCtx.memory().get(memPtr)) {
- BinaryRawReaderEx reader = platformCtx.reader(mem);
-
- return processInStreamOutLong(type, reader, mem);
- }
- catch (Exception e) {
- throw convertException(e);
- }
- }
-
- /** {@inheritDoc} */
- @Override public Object inStreamOutObject(int type, long memPtr) throws Exception {
- try (PlatformMemory mem = memPtr != 0 ? platformCtx.memory().get(memPtr) : null) {
- BinaryRawReaderEx reader = mem != null ? platformCtx.reader(mem) : null;
-
- return processInStreamOutObject(type, reader);
- }
- catch (Exception e) {
- throw convertException(e);
- }
- }
-
- /** {@inheritDoc} */
- @Override public void outStream(int type, long memPtr) throws Exception {
- try (PlatformMemory mem = platformCtx.memory().get(memPtr)) {
- PlatformOutputStream out = mem.output();
-
- BinaryRawWriterEx writer = platformCtx.writer(out);
-
- processOutStream(type, writer);
-
- out.synchronize();
- }
- catch (Exception e) {
- throw convertException(e);
- }
- }
-
- /** {@inheritDoc} */
- @Override public Object outObject(int type) throws Exception {
- try {
- return processOutObject(type);
- }
- catch (Exception e) {
- throw convertException(e);
- }
- }
-
- /** {@inheritDoc} */
- @Override public void inStreamOutStream(int type, long inMemPtr, long outMemPtr) throws Exception {
- try (PlatformMemory inMem = platformCtx.memory().get(inMemPtr)) {
- BinaryRawReaderEx reader = platformCtx.reader(inMem);
-
- try (PlatformMemory outMem = platformCtx.memory().get(outMemPtr)) {
- PlatformOutputStream out = outMem.output();
-
- BinaryRawWriterEx writer = platformCtx.writer(out);
-
- processInStreamOutStream(type, reader, writer);
-
- out.synchronize();
- }
- }
- catch (Exception e) {
- throw convertException(e);
- }
- }
-
- /** {@inheritDoc} */
- @Override public Object inObjectStreamOutObjectStream(int type, Object arg, long inMemPtr, long outMemPtr)
- throws Exception {
- PlatformMemory inMem = null;
- PlatformMemory outMem = null;
-
- try {
- BinaryRawReaderEx reader = null;
-
- if (inMemPtr != 0) {
- inMem = platformCtx.memory().get(inMemPtr);
-
- reader = platformCtx.reader(inMem);
- }
-
- PlatformOutputStream out = null;
- BinaryRawWriterEx writer = null;
-
- if (outMemPtr != 0) {
- outMem = platformCtx.memory().get(outMemPtr);
-
- out = outMem.output();
-
- writer = platformCtx.writer(out);
- }
-
- Object res = processInObjectStreamOutObjectStream(type, arg, reader, writer);
-
- if (out != null)
- out.synchronize();
-
- return res;
- }
- catch (Exception e) {
- throw convertException(e);
- }
- finally {
- try {
- if (inMem != null)
- inMem.close();
- }
- finally {
- if (outMem != null)
- outMem.close();
- }
- }
- }
-
- /**
- * Convert caught exception.
- *
- * @param e Exception to convert.
- * @return Converted exception.
- */
- public Exception convertException(Exception e) {
- return e;
- }
-
/**
* @return Context.
*/
@@ -206,128 +68,60 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
}
/** {@inheritDoc} */
- @Override public void listenFuture(final long futId, int typ) throws Exception {
- PlatformFutureUtils.listen(platformCtx, currentFuture(), futId, typ, null, this);
+ @Override public Exception convertException(Exception e) {
+ return e;
}
/** {@inheritDoc} */
- @Override public void listenFutureForOperation(final long futId, int typ, int opId) throws Exception {
- PlatformFutureUtils.listen(platformCtx, currentFuture(), futId, typ, futureWriter(opId), this);
- }
-
- /**
- * When overridden in a derived class, gets future for the current operation.
- *
- * @return current future.
- * @throws IgniteCheckedException If failed.
- */
- protected IgniteInternalFuture currentFuture() throws IgniteCheckedException {
+ @Override public IgniteInternalFuture currentFuture() throws IgniteCheckedException {
throw new IgniteCheckedException("Future listening is not supported in " + getClass());
}
- /**
- * When overridden in a derived class, gets a custom future writer.
- *
- * @param opId Operation id.
- * @return A custom writer for given op id.
- */
- @Nullable protected PlatformFutureUtils.Writer futureWriter(int opId){
+ /** {@inheritDoc} */
+ @Override @Nullable public PlatformFutureUtils.Writer futureWriter(int opId){
return null;
}
- /**
- * Process IN operation.
- *
- * @param type Type.
- * @param val Value.
- * @return Result.
- * @throws IgniteCheckedException In case of exception.
- */
- protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+ /** {@inheritDoc} */
+ @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
return throwUnsupported(type);
}
- /**
- * Process IN operation.
- *
- * @param type Type.
- * @param reader Binary reader.
- * @return Result.
- * @throws IgniteCheckedException In case of exception.
- */
- protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+ /** {@inheritDoc} */
+ @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
return throwUnsupported(type);
}
- /**
- * Process IN operation.
- *
- * @param type Type.
- * @param reader Binary reader.
- * @return Result.
- * @throws IgniteCheckedException In case of exception.
- */
- protected long processInStreamOutLong(int type, BinaryRawReaderEx reader, PlatformMemory mem) throws IgniteCheckedException {
+ /** {@inheritDoc} */
+ @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader, PlatformMemory mem) throws IgniteCheckedException {
return processInStreamOutLong(type, reader);
}
- /**
- * Process IN-OUT operation.
- *
- * @param type Type.
- * @param reader Binary reader.
- * @param writer Binary writer.
- * @throws IgniteCheckedException In case of exception.
- */
- protected void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
+ /** {@inheritDoc} */
+ @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
throws IgniteCheckedException {
throwUnsupported(type);
}
- /**
- * Process IN operation with managed object as result.
- *
- * @param type Type.
- * @param reader Binary reader.
- * @return Result.
- * @throws IgniteCheckedException In case of exception.
- */
- protected Object processInStreamOutObject(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+ /** {@inheritDoc} */
+ @Override public PlatformTarget processInStreamOutObject(int type, BinaryRawReaderEx reader)
+ throws IgniteCheckedException {
return throwUnsupported(type);
}
- /**
- * Process IN-OUT operation.
- *
- * @param type Type.
- * @param arg Argument.
- * @param reader Binary reader.
- * @param writer Binary writer.
- * @throws IgniteCheckedException In case of exception.
- */
- protected Object processInObjectStreamOutObjectStream(int type, @Nullable Object arg, BinaryRawReaderEx reader,
- BinaryRawWriterEx writer) throws IgniteCheckedException {
+ /** {@inheritDoc} */
+ @Override public PlatformTarget processInObjectStreamOutObjectStream(int type, @Nullable PlatformTarget arg,
+ BinaryRawReaderEx reader, BinaryRawWriterEx writer) throws IgniteCheckedException {
return throwUnsupported(type);
}
- /**
- * Process OUT operation.
- *
- * @param type Type.
- * @param writer Binary writer.
- * @throws IgniteCheckedException In case of exception.
- */
- protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+ /** {@inheritDoc} */
+ @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
throwUnsupported(type);
}
- /**
- * Process OUT operation.
- *
- * @param type Type.
- * @throws IgniteCheckedException In case of exception.
- */
- protected Object processOutObject(int type) throws IgniteCheckedException {
+ /** {@inheritDoc} */
+ @Override public PlatformTarget processOutObject(int type) throws IgniteCheckedException {
return throwUnsupported(type);
}
@@ -338,7 +132,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
* @return Dummy value which is never returned.
* @throws IgniteCheckedException Exception to be thrown.
*/
- protected <T> T throwUnsupported(int type) throws IgniteCheckedException {
+ private <T> T throwUnsupported(int type) throws IgniteCheckedException {
throw new IgniteCheckedException("Unsupported operation type: " + type);
}
@@ -411,4 +205,14 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
return TRUE;
}
+
+ /**
+ * Wraps a listenable to be returned to platform.
+ *
+ * @param listenable Listenable.
+ * @return Target.
+ */
+ protected PlatformTarget wrapListenable(PlatformListenable listenable) {
+ return new PlatformListenableTarget(listenable, platformCtx);
+ }
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAsyncTarget.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAsyncTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAsyncTarget.java
new file mode 100644
index 0000000..a4d35c9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAsyncTarget.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Async target.
+ */
+public interface PlatformAsyncTarget {
+ /**
+ * Gets future for the current operation.
+ *
+ * @return current future.
+ * @throws IgniteCheckedException If failed.
+ */
+ IgniteInternalFuture currentFuture() throws IgniteCheckedException;
+
+ /**
+ * Gets a custom future writer.
+ *
+ * @param opId Operation id.
+ * @return A custom writer for given op id.
+ */
+ @Nullable PlatformFutureUtils.Writer futureWriter(int opId);
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
index fd357ec..2911418 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
@@ -61,27 +61,27 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
}
/** {@inheritDoc} */
- @Override public PlatformTarget cache(@Nullable String name) throws IgniteCheckedException {
+ @Override public PlatformTargetProxy cache(@Nullable String name) throws IgniteCheckedException {
return null;
}
/** {@inheritDoc} */
- @Override public PlatformTarget createCache(@Nullable String name) throws IgniteCheckedException {
+ @Override public PlatformTargetProxy createCache(@Nullable String name) throws IgniteCheckedException {
return null;
}
/** {@inheritDoc} */
- @Override public PlatformTarget getOrCreateCache(@Nullable String name) throws IgniteCheckedException {
+ @Override public PlatformTargetProxy getOrCreateCache(@Nullable String name) throws IgniteCheckedException {
return null;
}
/** {@inheritDoc} */
- @Override public PlatformTarget createCacheFromConfig(long memPtr) throws IgniteCheckedException {
+ @Override public PlatformTargetProxy createCacheFromConfig(long memPtr) throws IgniteCheckedException {
return null;
}
/** {@inheritDoc} */
- @Override public PlatformTarget getOrCreateCacheFromConfig(long memPtr) throws IgniteCheckedException {
+ @Override public PlatformTargetProxy getOrCreateCacheFromConfig(long memPtr) throws IgniteCheckedException {
return null;
}
@@ -91,47 +91,48 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
}
/** {@inheritDoc} */
- @Override public PlatformTarget affinity(@Nullable String name) throws IgniteCheckedException {
+ @Override public PlatformTargetProxy affinity(@Nullable String name) throws IgniteCheckedException {
return null;
}
/** {@inheritDoc} */
- @Override public PlatformTarget dataStreamer(@Nullable String cacheName, boolean keepBinary) throws IgniteCheckedException {
+ @Override public PlatformTargetProxy dataStreamer(@Nullable String cacheName, boolean keepBinary)
+ throws IgniteCheckedException {
return null;
}
/** {@inheritDoc} */
- @Override public PlatformTarget transactions() {
+ @Override public PlatformTargetProxy transactions() {
return null;
}
/** {@inheritDoc} */
- @Override public PlatformTarget projection() throws IgniteCheckedException {
+ @Override public PlatformTargetProxy projection() throws IgniteCheckedException {
return null;
}
/** {@inheritDoc} */
- @Override public PlatformTarget compute(PlatformTarget grp) {
+ @Override public PlatformTargetProxy compute(PlatformTargetProxy grp) {
return null;
}
/** {@inheritDoc} */
- @Override public PlatformTarget message(PlatformTarget grp) {
+ @Override public PlatformTargetProxy message(PlatformTargetProxy grp) {
return null;
}
/** {@inheritDoc} */
- @Override public PlatformTarget events(PlatformTarget grp) {
+ @Override public PlatformTargetProxy events(PlatformTargetProxy grp) {
return null;
}
/** {@inheritDoc} */
- @Override public PlatformTarget services(PlatformTarget grp) {
+ @Override public PlatformTargetProxy services(PlatformTargetProxy grp) {
return null;
}
/** {@inheritDoc} */
- @Override public PlatformTarget extensions() {
+ @Override public PlatformTargetProxy extensions() {
return null;
}
@@ -142,7 +143,7 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
}
/** {@inheritDoc} */
- @Override public PlatformTarget atomicLong(String name, long initVal, boolean create) throws IgniteException {
+ @Override public PlatformTargetProxy atomicLong(String name, long initVal, boolean create) throws IgniteException {
return null;
}
@@ -157,22 +158,22 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
}
/** {@inheritDoc} */
- @Override public PlatformTarget atomicSequence(String name, long initVal, boolean create) throws IgniteException {
+ @Override public PlatformTargetProxy atomicSequence(String name, long initVal, boolean create) throws IgniteException {
return null;
}
/** {@inheritDoc} */
- @Override public PlatformTarget atomicReference(String name, long memPtr, boolean create) throws IgniteException {
+ @Override public PlatformTargetProxy atomicReference(String name, long memPtr, boolean create) throws IgniteException {
return null;
}
/** {@inheritDoc} */
- @Override public PlatformTarget createNearCache(@Nullable String cacheName, long memPtr) {
+ @Override public PlatformTargetProxy createNearCache(@Nullable String cacheName, long memPtr) {
return null;
}
/** {@inheritDoc} */
- @Override public PlatformTarget getOrCreateNearCache(@Nullable String cacheName, long memPtr) {
+ @Override public PlatformTargetProxy getOrCreateNearCache(@Nullable String cacheName, long memPtr) {
return null;
}
@@ -187,7 +188,7 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
}
/** {@inheritDoc} */
- @Override public PlatformTarget binaryProcessor() {
+ @Override public PlatformTargetProxy binaryProcessor() {
return null;
}
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
index f01175e..e0d94d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
@@ -26,7 +26,7 @@ import org.jetbrains.annotations.Nullable;
/**
* Platform processor.
*/
-@SuppressWarnings("UnusedDeclaration")
+@SuppressWarnings({"UnusedDeclaration", "UnnecessaryInterfaceModifier"})
public interface PlatformProcessor extends GridProcessor {
/**
* Gets owning Ignite instance.
@@ -68,7 +68,7 @@ public interface PlatformProcessor extends GridProcessor {
* @return Cache.
* @throws IgniteCheckedException If failed.
*/
- public PlatformTarget cache(@Nullable String name) throws IgniteCheckedException;
+ public PlatformTargetProxy cache(@Nullable String name) throws IgniteCheckedException;
/**
* Create cache.
@@ -77,7 +77,7 @@ public interface PlatformProcessor extends GridProcessor {
* @return Cache.
* @throws IgniteCheckedException If failed.
*/
- public PlatformTarget createCache(@Nullable String name) throws IgniteCheckedException;
+ public PlatformTargetProxy createCache(@Nullable String name) throws IgniteCheckedException;
/**
* Get or create cache.
@@ -86,7 +86,7 @@ public interface PlatformProcessor extends GridProcessor {
* @return Cache.
* @throws IgniteCheckedException If failed.
*/
- public PlatformTarget getOrCreateCache(@Nullable String name) throws IgniteCheckedException;
+ public PlatformTargetProxy getOrCreateCache(@Nullable String name) throws IgniteCheckedException;
/**
* Create cache.
@@ -95,7 +95,7 @@ public interface PlatformProcessor extends GridProcessor {
* @return Cache.
* @throws IgniteCheckedException If failed.
*/
- public PlatformTarget createCacheFromConfig(long memPtr) throws IgniteCheckedException;
+ public PlatformTargetProxy createCacheFromConfig(long memPtr) throws IgniteCheckedException;
/**
* Get or create cache.
@@ -104,7 +104,7 @@ public interface PlatformProcessor extends GridProcessor {
* @return Cache.
* @throws IgniteCheckedException If failed.
*/
- public PlatformTarget getOrCreateCacheFromConfig(long memPtr) throws IgniteCheckedException;
+ public PlatformTargetProxy getOrCreateCacheFromConfig(long memPtr) throws IgniteCheckedException;
/**
* Destroy dynamically created cache.
@@ -121,7 +121,7 @@ public interface PlatformProcessor extends GridProcessor {
* @return Affinity.
* @throws IgniteCheckedException If failed.
*/
- public PlatformTarget affinity(@Nullable String name) throws IgniteCheckedException;
+ public PlatformTargetProxy affinity(@Nullable String name) throws IgniteCheckedException;
/**
* Get data streamer.
@@ -131,14 +131,14 @@ public interface PlatformProcessor extends GridProcessor {
* @return Data streamer.
* @throws IgniteCheckedException If failed.
*/
- public PlatformTarget dataStreamer(@Nullable String cacheName, boolean keepBinary) throws IgniteCheckedException;
+ public PlatformTargetProxy dataStreamer(@Nullable String cacheName, boolean keepBinary) throws IgniteCheckedException;
/**
* Get transactions.
*
* @return Transactions.
*/
- public PlatformTarget transactions();
+ public PlatformTargetProxy transactions();
/**
* Get projection.
@@ -146,7 +146,7 @@ public interface PlatformProcessor extends GridProcessor {
* @return Projection.
* @throws IgniteCheckedException If failed.
*/
- public PlatformTarget projection() throws IgniteCheckedException;
+ public PlatformTargetProxy projection() throws IgniteCheckedException;
/**
* Create interop compute.
@@ -154,7 +154,7 @@ public interface PlatformProcessor extends GridProcessor {
* @param grp Cluster group.
* @return Compute instance.
*/
- public PlatformTarget compute(PlatformTarget grp);
+ public PlatformTargetProxy compute(PlatformTargetProxy grp);
/**
* Create interop messaging.
@@ -162,7 +162,7 @@ public interface PlatformProcessor extends GridProcessor {
* @param grp Cluster group.
* @return Messaging instance.
*/
- public PlatformTarget message(PlatformTarget grp);
+ public PlatformTargetProxy message(PlatformTargetProxy grp);
/**
* Create interop events.
@@ -170,7 +170,7 @@ public interface PlatformProcessor extends GridProcessor {
* @param grp Cluster group.
* @return Events instance.
*/
- public PlatformTarget events(PlatformTarget grp);
+ public PlatformTargetProxy events(PlatformTargetProxy grp);
/**
* Create interop services.
@@ -178,14 +178,14 @@ public interface PlatformProcessor extends GridProcessor {
* @param grp Cluster group.
* @return Services instance.
*/
- public PlatformTarget services(PlatformTarget grp);
+ public PlatformTargetProxy services(PlatformTargetProxy grp);
/**
* Get platform extensions. Override this method to provide any additional targets and operations you need.
*
* @return Platform extensions.
*/
- public PlatformTarget extensions();
+ public PlatformTargetProxy extensions();
/**
* Register cache store.
@@ -203,7 +203,7 @@ public interface PlatformProcessor extends GridProcessor {
* @param create Create flag.
* @return Platform atomic long.
*/
- public PlatformTarget atomicLong(String name, long initVal, boolean create);
+ public PlatformTargetProxy atomicLong(String name, long initVal, boolean create);
/**
* Get or create AtomicSequence.
@@ -212,7 +212,7 @@ public interface PlatformProcessor extends GridProcessor {
* @param create Create flag.
* @return Platform atomic long.
*/
- public PlatformTarget atomicSequence(String name, long initVal, boolean create);
+ public PlatformTargetProxy atomicSequence(String name, long initVal, boolean create);
/**
* Get or create AtomicReference.
@@ -221,7 +221,7 @@ public interface PlatformProcessor extends GridProcessor {
* @param create Create flag.
* @return Platform atomic long.
*/
- public PlatformTarget atomicReference(String name, long memPtr, boolean create);
+ public PlatformTargetProxy atomicReference(String name, long memPtr, boolean create);
/**
* Gets the configuration of the current Ignite instance.
@@ -244,7 +244,7 @@ public interface PlatformProcessor extends GridProcessor {
* @param memPtr Pointer to a stream with near cache config. 0 for default config.
* @return Cache.
*/
- public PlatformTarget createNearCache(@Nullable String cacheName, long memPtr);
+ public PlatformTargetProxy createNearCache(@Nullable String cacheName, long memPtr);
/**
* Gets existing near cache with the given name or creates a new one.
@@ -253,7 +253,7 @@ public interface PlatformProcessor extends GridProcessor {
* @param memPtr Pointer to a stream with near cache config. 0 for default config.
* @return Cache.
*/
- public PlatformTarget getOrCreateNearCache(@Nullable String cacheName, long memPtr);
+ public PlatformTargetProxy getOrCreateNearCache(@Nullable String cacheName, long memPtr);
/**
* Gets a value indicating whether Ignite logger has specified level enabled.
@@ -277,5 +277,5 @@ public interface PlatformProcessor extends GridProcessor {
*
* @return Binary processor.
*/
- public PlatformTarget binaryProcessor();
+ public PlatformTargetProxy binaryProcessor();
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
index f775987..8c81ebb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
@@ -220,7 +220,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
}
/** {@inheritDoc} */
- @Override public PlatformTarget cache(@Nullable String name) throws IgniteCheckedException {
+ @Override public PlatformTargetProxy cache(@Nullable String name) throws IgniteCheckedException {
IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().cache(name);
if (cache == null)
@@ -230,7 +230,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
}
/** {@inheritDoc} */
- @Override public PlatformTarget createCache(@Nullable String name) throws IgniteCheckedException {
+ @Override public PlatformTargetProxy createCache(@Nullable String name) throws IgniteCheckedException {
IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().createCache(name);
assert cache != null;
@@ -239,7 +239,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
}
/** {@inheritDoc} */
- @Override public PlatformTarget getOrCreateCache(@Nullable String name) throws IgniteCheckedException {
+ @Override public PlatformTargetProxy getOrCreateCache(@Nullable String name) throws IgniteCheckedException {
IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().getOrCreateCache(name);
assert cache != null;
@@ -248,7 +248,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
}
/** {@inheritDoc} */
- @Override public PlatformTarget createCacheFromConfig(long memPtr) throws IgniteCheckedException {
+ @Override public PlatformTargetProxy createCacheFromConfig(long memPtr) throws IgniteCheckedException {
BinaryRawReaderEx reader = platformCtx.reader(platformCtx.memory().get(memPtr));
CacheConfiguration cfg = PlatformConfigurationUtils.readCacheConfiguration(reader);
@@ -260,7 +260,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
}
/** {@inheritDoc} */
- @Override public PlatformTarget getOrCreateCacheFromConfig(long memPtr) throws IgniteCheckedException {
+ @Override public PlatformTargetProxy getOrCreateCacheFromConfig(long memPtr) throws IgniteCheckedException {
BinaryRawReaderEx reader = platformCtx.reader(platformCtx.memory().get(memPtr));
CacheConfiguration cfg = PlatformConfigurationUtils.readCacheConfiguration(reader);
@@ -278,60 +278,60 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
}
/** {@inheritDoc} */
- @Override public PlatformTarget affinity(@Nullable String name) throws IgniteCheckedException {
- return new PlatformAffinity(platformCtx, ctx, name);
+ @Override public PlatformTargetProxy affinity(@Nullable String name) throws IgniteCheckedException {
+ return proxy(new PlatformAffinity(platformCtx, ctx, name));
}
/** {@inheritDoc} */
- @Override public PlatformTarget dataStreamer(@Nullable String cacheName, boolean keepBinary)
+ @Override public PlatformTargetProxy dataStreamer(@Nullable String cacheName, boolean keepBinary)
throws IgniteCheckedException {
IgniteDataStreamer ldr = ctx.dataStream().dataStreamer(cacheName);
ldr.keepBinary(true);
- return new PlatformDataStreamer(platformCtx, cacheName, (DataStreamerImpl)ldr, keepBinary);
+ return proxy(new PlatformDataStreamer(platformCtx, cacheName, (DataStreamerImpl)ldr, keepBinary));
}
/** {@inheritDoc} */
- @Override public PlatformTarget transactions() {
- return new PlatformTransactions(platformCtx);
+ @Override public PlatformTargetProxy transactions() {
+ return proxy(new PlatformTransactions(platformCtx));
}
/** {@inheritDoc} */
- @Override public PlatformTarget projection() throws IgniteCheckedException {
- return new PlatformClusterGroup(platformCtx, ctx.grid().cluster());
+ @Override public PlatformTargetProxy projection() throws IgniteCheckedException {
+ return proxy(new PlatformClusterGroup(platformCtx, ctx.grid().cluster()));
}
/** {@inheritDoc} */
- @Override public PlatformTarget compute(PlatformTarget grp) {
- PlatformClusterGroup grp0 = (PlatformClusterGroup)grp;
+ @Override public PlatformTargetProxy compute(PlatformTargetProxy grp) {
+ PlatformClusterGroup grp0 = (PlatformClusterGroup)grp.unwrap();
- return new PlatformCompute(platformCtx, grp0.projection(), PlatformUtils.ATTR_PLATFORM);
+ return proxy(new PlatformCompute(platformCtx, grp0.projection(), PlatformUtils.ATTR_PLATFORM));
}
/** {@inheritDoc} */
- @Override public PlatformTarget message(PlatformTarget grp) {
- PlatformClusterGroup grp0 = (PlatformClusterGroup)grp;
+ @Override public PlatformTargetProxy message(PlatformTargetProxy grp) {
+ PlatformClusterGroup grp0 = (PlatformClusterGroup)grp.unwrap();
- return new PlatformMessaging(platformCtx, grp0.projection().ignite().message(grp0.projection()));
+ return proxy(new PlatformMessaging(platformCtx, grp0.projection().ignite().message(grp0.projection())));
}
/** {@inheritDoc} */
- @Override public PlatformTarget events(PlatformTarget grp) {
- PlatformClusterGroup grp0 = (PlatformClusterGroup)grp;
+ @Override public PlatformTargetProxy events(PlatformTargetProxy grp) {
+ PlatformClusterGroup grp0 = (PlatformClusterGroup)grp.unwrap();
- return new PlatformEvents(platformCtx, grp0.projection().ignite().events(grp0.projection()));
+ return proxy(new PlatformEvents(platformCtx, grp0.projection().ignite().events(grp0.projection())));
}
/** {@inheritDoc} */
- @Override public PlatformTarget services(PlatformTarget grp) {
- PlatformClusterGroup grp0 = (PlatformClusterGroup)grp;
+ @Override public PlatformTargetProxy services(PlatformTargetProxy grp) {
+ PlatformClusterGroup grp0 = (PlatformClusterGroup)grp.unwrap();
- return new PlatformServices(platformCtx, grp0.projection().ignite().services(grp0.projection()), false);
+ return proxy(new PlatformServices(platformCtx, grp0.projection().ignite().services(grp0.projection()), false));
}
/** {@inheritDoc} */
- @Override public PlatformTarget extensions() {
+ @Override public PlatformTargetProxy extensions() {
return null;
}
@@ -356,28 +356,32 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
}
/** {@inheritDoc} */
- @Override public PlatformTarget atomicLong(String name, long initVal, boolean create) throws IgniteException {
+ @Override public PlatformTargetProxy atomicLong(String name, long initVal, boolean create) throws IgniteException {
GridCacheAtomicLongImpl atomicLong = (GridCacheAtomicLongImpl)ignite().atomicLong(name, initVal, create);
if (atomicLong == null)
return null;
- return new PlatformAtomicLong(platformCtx, atomicLong);
+ return proxy(new PlatformAtomicLong(platformCtx, atomicLong));
}
/** {@inheritDoc} */
- @Override public PlatformTarget atomicSequence(String name, long initVal, boolean create) throws IgniteException {
+ @Override public PlatformTargetProxy atomicSequence(String name, long initVal, boolean create)
+ throws IgniteException {
IgniteAtomicSequence atomicSeq = ignite().atomicSequence(name, initVal, create);
if (atomicSeq == null)
return null;
- return new PlatformAtomicSequence(platformCtx, atomicSeq);
+ return proxy(new PlatformAtomicSequence(platformCtx, atomicSeq));
}
/** {@inheritDoc} */
- @Override public PlatformTarget atomicReference(String name, long memPtr, boolean create) throws IgniteException {
- return PlatformAtomicReference.createInstance(platformCtx, name, memPtr, create);
+ @Override public PlatformTargetProxy atomicReference(String name, long memPtr, boolean create)
+ throws IgniteException {
+ PlatformAtomicReference ref = PlatformAtomicReference.createInstance(platformCtx, name, memPtr, create);
+
+ return ref != null ? proxy(ref) : null;
}
/** {@inheritDoc} */
@@ -427,7 +431,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
}
/** {@inheritDoc} */
- @Override public PlatformTarget createNearCache(@Nullable String cacheName, long memPtr) {
+ @Override public PlatformTargetProxy createNearCache(@Nullable String cacheName, long memPtr) {
NearCacheConfiguration cfg = getNearCacheConfiguration(memPtr);
IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().createNearCache(cacheName, cfg);
@@ -436,7 +440,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
}
/** {@inheritDoc} */
- @Override public PlatformTarget getOrCreateNearCache(@Nullable String cacheName, long memPtr) {
+ @Override public PlatformTargetProxy getOrCreateNearCache(@Nullable String cacheName, long memPtr) {
NearCacheConfiguration cfg = getNearCacheConfiguration(memPtr);
IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().getOrCreateNearCache(cacheName, cfg);
@@ -447,8 +451,8 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
/**
* Creates new platform cache.
*/
- private PlatformTarget createPlatformCache(IgniteCacheProxy cache) {
- return new PlatformCache(platformCtx, cache, false, cacheExts);
+ private PlatformTargetProxy createPlatformCache(IgniteCacheProxy cache) {
+ return proxy(new PlatformCache(platformCtx, cache, false, cacheExts));
}
/** {@inheritDoc} */
@@ -504,8 +508,8 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
}
/** {@inheritDoc} */
- @Override public PlatformTarget binaryProcessor() {
- return new PlatformBinaryProcessor(platformCtx);
+ @Override public PlatformTargetProxy binaryProcessor() {
+ return proxy(new PlatformBinaryProcessor(platformCtx));
}
/**
@@ -580,6 +584,13 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
}
/**
+ * Wraps target in a proxy.
+ */
+ private PlatformTargetProxy proxy(PlatformTarget target) {
+ return new PlatformTargetProxyImpl(target, platformCtx);
+ }
+
+ /**
* Store and manager pair.
*/
private static class StoreInfo {
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
index 805fd5e..5d234dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
@@ -18,7 +18,9 @@
package org.apache.ignite.internal.processors.platform;
import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.processors.platform.utils.*;
+import org.apache.ignite.internal.binary.BinaryRawReaderEx;
+import org.apache.ignite.internal.binary.BinaryRawWriterEx;
+import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
import org.jetbrains.annotations.Nullable;
/**
@@ -27,94 +29,89 @@ import org.jetbrains.annotations.Nullable;
@SuppressWarnings("UnusedDeclaration")
public interface PlatformTarget {
/**
- * Operation accepting long value and returning long value.
+ * Process IN operation.
*
- * @param type Operation type.
+ * @param type Type.
* @param val Value.
* @return Result.
- * @throws Exception If case of failure.
+ * @throws IgniteCheckedException In case of exception.
*/
- public long inLongOutLong(int type, long val) throws Exception;
+ long processInLongOutLong(int type, long val) throws IgniteCheckedException;
/**
- * Operation accepting memory stream and returning long value.
+ * Process IN operation.
*
- * @param type Operation type.
- * @param memPtr Memory pointer.
+ * @param type Type.
+ * @param reader Binary reader.
* @return Result.
- * @throws Exception If case of failure.
+ * @throws IgniteCheckedException In case of exception.
*/
- public long inStreamOutLong(int type, long memPtr) throws Exception;
+ long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException;
/**
- * Operation accepting memory stream and returning object.
+ * Process IN operation.
*
- * @param type Operation type.
- * @param memPtr Memory pointer.
+ * @param type Type.
+ * @param reader Binary reader.
* @return Result.
- * @throws Exception If case of failure.
+ * @throws IgniteCheckedException In case of exception.
*/
- public Object inStreamOutObject(int type, long memPtr) throws Exception;
+ long processInStreamOutLong(int type, BinaryRawReaderEx reader, PlatformMemory mem) throws IgniteCheckedException;
/**
- * Operation accepting one memory stream and returning result to another memory stream.
+ * Process IN-OUT operation.
*
- * @param type Operation type.
- * @param inMemPtr Input memory pointer.
- * @param outMemPtr Output memory pointer.
- * @throws Exception In case of failure.
+ * @param type Type.
+ * @param reader Binary reader.
+ * @param writer Binary writer.
+ * @throws IgniteCheckedException In case of exception.
*/
- public void inStreamOutStream(int type, long inMemPtr, long outMemPtr) throws Exception;
+ void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
+ throws IgniteCheckedException;
/**
- * Operation accepting an object and a memory stream and returning result to another memory stream and an object.
+ * Process IN-OUT operation.
*
- * @param type Operation type.
- * @param arg Argument (optional).
- * @param inMemPtr Input memory pointer.
- * @param outMemPtr Output memory pointer.
- * @return Result.
- * @throws Exception In case of failure.
+ * @param type Type.
+ * @param reader Binary reader.
+ * @throws IgniteCheckedException In case of exception.
*/
- public Object inObjectStreamOutObjectStream(int type, @Nullable Object arg, long inMemPtr, long outMemPtr)
- throws Exception;
+ PlatformTarget processInStreamOutObject(int type, BinaryRawReaderEx reader) throws IgniteCheckedException;
/**
- * Operation returning result to memory stream.
+ * Process IN-OUT operation.
*
- * @param type Operation type.
- * @param memPtr Memory pointer.
- * @throws Exception In case of failure.
+ * @param type Type.
+ * @param arg Argument.
+ * @param reader Binary reader.
+ * @param writer Binary writer.
+ * @throws IgniteCheckedException In case of exception.
*/
- public void outStream(int type, long memPtr) throws Exception;
+ PlatformTarget processInObjectStreamOutObjectStream(int type, @Nullable PlatformTarget arg, BinaryRawReaderEx reader,
+ BinaryRawWriterEx writer) throws IgniteCheckedException;
/**
- * Operation returning object result.
+ * Process OUT operation.
*
- * @param type Operation type.
- * @return Result.
- * @throws Exception If failed.
+ * @param type Type.
+ * @param writer Binary writer.
+ * @throws IgniteCheckedException In case of exception.
*/
- public Object outObject(int type) throws Exception;
+ void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException;
/**
- * Start listening for the future.
+ * Process OUT operation.
*
- * @param futId Future ID.
- * @param typ Result type.
- * @throws IgniteCheckedException In case of failure.
+ * @param type Type.
+ * @throws IgniteCheckedException In case of exception.
*/
- @SuppressWarnings("UnusedDeclaration")
- public void listenFuture(final long futId, int typ) throws Exception;
+ PlatformTarget processOutObject(int type) throws IgniteCheckedException;
/**
- * Start listening for the future for specific operation type.
+ * Convert caught exception.
*
- * @param futId Future ID.
- * @param typ Result type.
- * @param opId Operation ID required to pick correct result writer.
- * @throws IgniteCheckedException In case of failure.
+ * @param e Exception to convert.
+ * @return Converted exception.
*/
- @SuppressWarnings("UnusedDeclaration")
- public void listenFutureForOperation(final long futId, int typ, int opId) throws Exception;
+ Exception convertException(Exception e);
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTargetProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTargetProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTargetProxy.java
new file mode 100644
index 0000000..a4f2a56
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTargetProxy.java
@@ -0,0 +1,126 @@
+/*
+ * 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.platform;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Platform target that is invoked via JNI and propagates calls to underlying {@link PlatformTarget}.
+ */
+@SuppressWarnings("UnusedDeclaration")
+public interface PlatformTargetProxy {
+ /**
+ * Operation accepting long value and returning long value.
+ *
+ * @param type Operation type.
+ * @param val Value.
+ * @return Result.
+ * @throws Exception If case of failure.
+ */
+ long inLongOutLong(int type, long val) throws Exception;
+
+ /**
+ * Operation accepting memory stream and returning long value.
+ *
+ * @param type Operation type.
+ * @param memPtr Memory pointer.
+ * @return Result.
+ * @throws Exception If case of failure.
+ */
+ long inStreamOutLong(int type, long memPtr) throws Exception;
+
+ /**
+ * Operation accepting memory stream and returning object.
+ *
+ * @param type Operation type.
+ * @param memPtr Memory pointer.
+ * @return Result.
+ * @throws Exception If case of failure.
+ */
+ Object inStreamOutObject(int type, long memPtr) throws Exception;
+
+ /**
+ * Operation accepting one memory stream and returning result to another memory stream.
+ *
+ * @param type Operation type.
+ * @param inMemPtr Input memory pointer.
+ * @param outMemPtr Output memory pointer.
+ * @throws Exception In case of failure.
+ */
+ void inStreamOutStream(int type, long inMemPtr, long outMemPtr) throws Exception;
+
+ /**
+ * Operation accepting an object and a memory stream and returning result to another memory stream and an object.
+ *
+ * @param type Operation type.
+ * @param arg Argument (optional).
+ * @param inMemPtr Input memory pointer.
+ * @param outMemPtr Output memory pointer.
+ * @return Result.
+ * @throws Exception In case of failure.
+ */
+ Object inObjectStreamOutObjectStream(int type, @Nullable Object arg, long inMemPtr, long outMemPtr)
+ throws Exception;
+
+ /**
+ * Operation returning result to memory stream.
+ *
+ * @param type Operation type.
+ * @param memPtr Memory pointer.
+ * @throws Exception In case of failure.
+ */
+ void outStream(int type, long memPtr) throws Exception;
+
+ /**
+ * Operation returning object result.
+ *
+ * @param type Operation type.
+ * @return Result.
+ * @throws Exception If failed.
+ */
+ Object outObject(int type) throws Exception;
+
+ /**
+ * Start listening for the future.
+ *
+ * @param futId Future ID.
+ * @param typ Result type.
+ * @throws IgniteCheckedException In case of failure.
+ */
+ @SuppressWarnings("UnusedDeclaration")
+ void listenFuture(final long futId, int typ) throws Exception;
+
+ /**
+ * Start listening for the future for specific operation type.
+ *
+ * @param futId Future ID.
+ * @param typ Result type.
+ * @param opId Operation ID required to pick correct result writer.
+ * @throws IgniteCheckedException In case of failure.
+ */
+ @SuppressWarnings("UnusedDeclaration")
+ void listenFutureForOperation(final long futId, int typ, int opId) throws Exception;
+
+ /**
+ * Returns the underlying target.
+ *
+ * @return Underlying target.
+ */
+ PlatformTarget unwrap();
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTargetProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTargetProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTargetProxyImpl.java
new file mode 100644
index 0000000..25a4de8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTargetProxyImpl.java
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.binary.BinaryRawReaderEx;
+import org.apache.ignite.internal.binary.BinaryRawWriterEx;
+import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
+import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
+import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils;
+
+/**
+ * Platform target that is invoked via JNI and propagates calls to underlying {@link PlatformTarget}.
+ */
+public class PlatformTargetProxyImpl implements PlatformTargetProxy {
+ /** Context. */
+ protected final PlatformContext platformCtx;
+
+ /** Underlying target. */
+ private final PlatformTarget target;
+
+ public PlatformTargetProxyImpl(PlatformTarget target, PlatformContext platformCtx) {
+ assert platformCtx != null;
+ assert target != null;
+
+ this.platformCtx = platformCtx;
+ this.target = target;
+ }
+
+ /** {@inheritDoc} */
+ @Override public long inLongOutLong(int type, long val) throws Exception {
+ try {
+ return target.processInLongOutLong(type, val);
+ }
+ catch (Exception e) {
+ throw target.convertException(e);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public long inStreamOutLong(int type, long memPtr) throws Exception {
+ try (PlatformMemory mem = platformCtx.memory().get(memPtr)) {
+ BinaryRawReaderEx reader = platformCtx.reader(mem);
+
+ return target.processInStreamOutLong(type, reader, mem);
+ }
+ catch (Exception e) {
+ throw target.convertException(e);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object inStreamOutObject(int type, long memPtr) throws Exception {
+ try (PlatformMemory mem = memPtr != 0 ? platformCtx.memory().get(memPtr) : null) {
+ BinaryRawReaderEx reader = mem != null ? platformCtx.reader(mem) : null;
+
+ return wrapProxy(target.processInStreamOutObject(type, reader));
+ }
+ catch (Exception e) {
+ throw target.convertException(e);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public void outStream(int type, long memPtr) throws Exception {
+ try (PlatformMemory mem = platformCtx.memory().get(memPtr)) {
+ PlatformOutputStream out = mem.output();
+
+ BinaryRawWriterEx writer = platformCtx.writer(out);
+
+ target.processOutStream(type, writer);
+
+ out.synchronize();
+ }
+ catch (Exception e) {
+ throw target.convertException(e);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object outObject(int type) throws Exception {
+ try {
+ return wrapProxy(target.processOutObject(type));
+ }
+ catch (Exception e) {
+ throw target.convertException(e);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public void inStreamOutStream(int type, long inMemPtr, long outMemPtr) throws Exception {
+ try (PlatformMemory inMem = platformCtx.memory().get(inMemPtr)) {
+ BinaryRawReaderEx reader = platformCtx.reader(inMem);
+
+ try (PlatformMemory outMem = platformCtx.memory().get(outMemPtr)) {
+ PlatformOutputStream out = outMem.output();
+
+ BinaryRawWriterEx writer = platformCtx.writer(out);
+
+ target.processInStreamOutStream(type, reader, writer);
+
+ out.synchronize();
+ }
+ }
+ catch (Exception e) {
+ throw target.convertException(e);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object inObjectStreamOutObjectStream(int type, Object arg, long inMemPtr, long outMemPtr)
+ throws Exception {
+ PlatformMemory inMem = null;
+ PlatformMemory outMem = null;
+
+ try {
+ BinaryRawReaderEx reader = null;
+
+ if (inMemPtr != 0) {
+ inMem = platformCtx.memory().get(inMemPtr);
+
+ reader = platformCtx.reader(inMem);
+ }
+
+ PlatformOutputStream out = null;
+ BinaryRawWriterEx writer = null;
+
+ if (outMemPtr != 0) {
+ outMem = platformCtx.memory().get(outMemPtr);
+
+ out = outMem.output();
+
+ writer = platformCtx.writer(out);
+ }
+
+ PlatformTarget res = target.processInObjectStreamOutObjectStream(type, unwrapProxy(arg), reader, writer);
+
+ if (out != null)
+ out.synchronize();
+
+ return wrapProxy(res);
+ }
+ catch (Exception e) {
+ throw target.convertException(e);
+ }
+ finally {
+ try {
+ if (inMem != null)
+ inMem.close();
+ }
+ finally {
+ if (outMem != null)
+ outMem.close();
+ }
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public void listenFuture(final long futId, int typ) throws Exception {
+ PlatformFutureUtils.listen(platformCtx, currentFuture(), futId, typ, null, target);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void listenFutureForOperation(final long futId, int typ, int opId) throws Exception {
+ PlatformFutureUtils.listen(platformCtx, currentFuture(), futId, typ, futureWriter(opId), target);
+ }
+
+ /** {@inheritDoc} */
+ @Override public PlatformTarget unwrap() {
+ return target;
+ }
+
+ /**
+ * @return Future writer.
+ */
+ private PlatformFutureUtils.Writer futureWriter(int opId) {
+ return ((PlatformAsyncTarget)target).futureWriter(opId);
+ }
+
+ /**
+ * @return Current future.
+ */
+ private IgniteInternalFuture currentFuture() throws IgniteCheckedException {
+ return ((PlatformAsyncTarget)target).currentFuture();
+ }
+
+ /**
+ * Wraps an object in a proxy when possible.
+ *
+ * @param obj Object to wrap.
+ * @return Wrapped object.
+ */
+ private Object wrapProxy(PlatformTarget obj) {
+ return obj == null ? null : new PlatformTargetProxyImpl(obj, platformCtx);
+ }
+
+ /**
+ * Unwraps an object from a proxy when possible.
+ *
+ * @param obj Object to unwrap.
+ * @return Unwrapped object.
+ */
+ private PlatformTarget unwrapProxy(Object obj) {
+ return obj == null ? null : ((PlatformTargetProxyImpl)obj).target;
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java
index 1bb577e..3c00abc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java
@@ -49,7 +49,7 @@ public class PlatformBinaryProcessor extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+ @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
if (type == OP_PUT_META) {
platformCtx.processMetadata(reader);
@@ -60,7 +60,7 @@ public class PlatformBinaryProcessor extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+ @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
if (type == OP_GET_ALL_META)
platformCtx.writeAllMetadata(writer);
else
@@ -68,7 +68,7 @@ public class PlatformBinaryProcessor extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader,
+ @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader,
BinaryRawWriterEx writer) throws IgniteCheckedException {
switch (type) {
case OP_GET_META: {
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
index aec3703..aee317e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
@@ -43,6 +43,7 @@ import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
import org.apache.ignite.internal.processors.platform.PlatformContext;
import org.apache.ignite.internal.processors.platform.PlatformNativeException;
import org.apache.ignite.internal.processors.platform.cache.expiry.PlatformExpiryPolicy;
+import org.apache.ignite.internal.processors.platform.PlatformTarget;
import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQuery;
import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryProxy;
import org.apache.ignite.internal.processors.platform.cache.query.PlatformFieldsQueryCursor;
@@ -400,7 +401,7 @@ public class PlatformCache extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader, PlatformMemory mem)
+ @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader, PlatformMemory mem)
throws IgniteCheckedException {
try {
switch (type) {
@@ -824,7 +825,7 @@ public class PlatformCache extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected Object processInStreamOutObject(int type, BinaryRawReaderEx reader)
+ @Override public PlatformTarget processInStreamOutObject(int type, BinaryRawReaderEx reader)
throws IgniteCheckedException {
switch (type) {
case OP_QRY_SQL:
@@ -903,7 +904,7 @@ public class PlatformCache extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+ @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
switch (type) {
case OP_GET_NAME:
writer.writeObject(cache.getName());
@@ -940,7 +941,7 @@ public class PlatformCache extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected Object processOutObject(int type) throws IgniteCheckedException {
+ @Override public PlatformTarget processOutObject(int type) throws IgniteCheckedException {
switch (type) {
case OP_WITH_ASYNC: {
if (cache.isAsync())
@@ -983,7 +984,7 @@ public class PlatformCache extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+ @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
switch (type) {
case OP_SIZE: {
CachePeekMode[] modes = PlatformUtils.decodeCachePeekModes((int)val);
@@ -1121,12 +1122,12 @@ public class PlatformCache extends PlatformAbstractTarget {
}
/** <inheritDoc /> */
- @Override protected IgniteInternalFuture currentFuture() throws IgniteCheckedException {
+ @Override public IgniteInternalFuture currentFuture() throws IgniteCheckedException {
return ((IgniteFutureImpl) cacheAsync.future()).internalFuture();
}
/** <inheritDoc /> */
- @Nullable @Override protected PlatformFutureUtils.Writer futureWriter(int opId) {
+ @Nullable @Override public PlatformFutureUtils.Writer futureWriter(int opId) {
if (opId == OP_GET_ALL)
return WRITER_GET_ALL;
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
index 292caea..4c11cc0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
@@ -47,7 +47,7 @@ public class PlatformCacheIterator extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+ @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
switch (type) {
case OP_NEXT:
if (iter.hasNext()) {
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
index 12df188..e24345c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
@@ -117,7 +117,7 @@ public class PlatformAffinity extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+ @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
switch (type) {
case OP_PARTITION:
return aff.partition(reader.readObjectDetached());
@@ -168,7 +168,7 @@ public class PlatformAffinity extends PlatformAbstractTarget {
/** {@inheritDoc} */
@SuppressWarnings({"IfMayBeConditional", "ConstantConditions"})
- @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
+ @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
throws IgniteCheckedException {
switch (type) {
case OP_PRIMARY_PARTITIONS: {
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunction.java
index 8076a19..2d3cada 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunction.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunction.java
@@ -26,6 +26,7 @@ import org.apache.ignite.cluster.ClusterNode;
import org.apache.ignite.internal.IgniteEx;
import org.apache.ignite.internal.binary.BinaryRawWriterEx;
import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformTargetProxyImpl;
import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
@@ -279,7 +280,11 @@ public class PlatformAffinityFunction implements AffinityFunction, Externalizabl
? new PlatformAffinityFunctionTarget(ctx, baseFunc)
: null;
- ptr = ctx.gateway().affinityFunctionInit(mem.pointer(), baseTarget);
+ PlatformTargetProxyImpl baseTargetProxy = baseTarget != null
+ ? new PlatformTargetProxyImpl(baseTarget, ctx)
+ : null;
+
+ ptr = ctx.gateway().affinityFunctionInit(mem.pointer(), baseTargetProxy);
}
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunctionTarget.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunctionTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunctionTarget.java
index 8a07b33..342e726 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunctionTarget.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunctionTarget.java
@@ -71,7 +71,7 @@ public class PlatformAffinityFunctionTarget extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+ @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
if (type == OP_PARTITION)
return baseFunc.partition(reader.readObjectDetached());
else if (type == OP_REMOVE_NODE) {
@@ -84,7 +84,7 @@ public class PlatformAffinityFunctionTarget extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+ @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
if (type == OP_ASSIGN_PARTITIONS) {
AffinityFunctionContext affCtx = currentAffCtx.get();
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
index 6a259ca..f201425 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
@@ -71,7 +71,7 @@ public abstract class PlatformAbstractQueryCursor<T> extends PlatformAbstractTar
}
/** {@inheritDoc} */
- @Override protected void processOutStream(int type, final BinaryRawWriterEx writer) throws IgniteCheckedException {
+ @Override public void processOutStream(int type, final BinaryRawWriterEx writer) throws IgniteCheckedException {
switch (type) {
case OP_GET_BATCH: {
assert iter != null : "iterator() has not been called";
@@ -136,7 +136,7 @@ public abstract class PlatformAbstractQueryCursor<T> extends PlatformAbstractTar
}
/** {@inheritDoc} */
- @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+ @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
switch (type) {
case OP_ITERATOR:
iter = cursor.iterator();
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java
index 04f17ff..27d784a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.platform.cache.query;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformTarget;
/**
* Proxy that implements PlatformTarget.
@@ -41,7 +42,7 @@ public class PlatformContinuousQueryProxy extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override public Object outObject(int type) throws Exception {
+ @Override public PlatformTarget processOutObject(int type) throws IgniteCheckedException {
return qry.getInitialQueryCursor();
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
index f21861e..c77f501 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.platform.callback;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteException;
-import org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinityFunctionTarget;
+import org.apache.ignite.internal.processors.platform.PlatformTargetProxy;
import org.apache.ignite.internal.util.GridStripedSpinBusyLock;
/**
@@ -429,7 +429,7 @@ public class PlatformCallbackGateway {
* @param memPtr Stream pointer.
* @param keepBinary Binary flag.
*/
- public void dataStreamerStreamReceiverInvoke(long ptr, Object cache, long memPtr, boolean keepBinary) {
+ public void dataStreamerStreamReceiverInvoke(long ptr, PlatformTargetProxy cache, long memPtr, boolean keepBinary) {
enter();
try {
@@ -995,7 +995,7 @@ public class PlatformCallbackGateway {
* @param baseFunc Optional func for base calls.
* @return Affinity function pointer.
*/
- public long affinityFunctionInit(long memPtr, PlatformAffinityFunctionTarget baseFunc) {
+ public long affinityFunctionInit(long memPtr, PlatformTargetProxy baseFunc) {
enter();
try {
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
index 50c4c28..9d60ec0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
@@ -17,7 +17,7 @@
package org.apache.ignite.internal.processors.platform.callback;
-import org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinityFunctionTarget;
+import org.apache.ignite.internal.processors.platform.PlatformTargetProxy;
/**
* Platform callback utility methods. Implemented in target platform. All methods in this class must be
@@ -226,7 +226,7 @@ public class PlatformCallbackUtils {
* @param memPtr Stream pointer.
* @param keepBinary Binary flag.
*/
- static native void dataStreamerStreamReceiverInvoke(long envPtr, long ptr, Object cache, long memPtr,
+ static native void dataStreamerStreamReceiverInvoke(long envPtr, long ptr, PlatformTargetProxy cache, long memPtr,
boolean keepBinary);
/**
@@ -504,7 +504,7 @@ public class PlatformCallbackUtils {
* @param baseFunc Optional func for base calls.
* @return Affinity function pointer.
*/
- static native long affinityFunctionInit(long envPtr, long memPtr, PlatformAffinityFunctionTarget baseFunc);
+ static native long affinityFunctionInit(long envPtr, long memPtr, PlatformTargetProxy baseFunc);
/**
* Gets the partition from affinity function.
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
index dc73468..f49f477 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx;
import org.apache.ignite.internal.binary.BinaryRawWriterEx;
import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformTarget;
import org.apache.ignite.internal.processors.platform.cache.PlatformCache;
import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
@@ -120,7 +121,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
/** {@inheritDoc} */
@SuppressWarnings("deprecation")
- @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+ @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
switch (type) {
case OP_METRICS:
platformCtx.writeClusterMetrics(writer, prj.metrics());
@@ -134,7 +135,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
/** {@inheritDoc} */
@SuppressWarnings({"ConstantConditions", "deprecation"})
- @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
+ @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
throws IgniteCheckedException {
switch (type) {
case OP_METRICS_FILTERED: {
@@ -217,7 +218,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+ @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
switch (type) {
case OP_PING_NODE:
return pingNode(reader.readUuid()) ? TRUE : FALSE;
@@ -228,7 +229,8 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected Object processInStreamOutObject(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+ @Override public PlatformTarget processInStreamOutObject(int type, BinaryRawReaderEx reader)
+ throws IgniteCheckedException {
switch (type) {
case OP_FOR_NODE_IDS: {
Collection<UUID> ids = PlatformUtils.readCollection(reader);
@@ -272,8 +274,8 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected Object processInObjectStreamOutObjectStream(
- int type, @Nullable Object arg, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
+ @Override public PlatformTarget processInObjectStreamOutObjectStream(
+ int type, @Nullable PlatformTarget arg, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
throws IgniteCheckedException {
switch (type) {
case OP_FOR_OTHERS: {
@@ -289,7 +291,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected Object processOutObject(int type) throws IgniteCheckedException {
+ @Override public PlatformTarget processOutObject(int type) throws IgniteCheckedException {
switch (type) {
case OP_FOR_REMOTES:
return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forRemotes());
@@ -314,7 +316,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+ @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
switch (type) {
case OP_RESET_METRICS: {
assert prj instanceof IgniteCluster; // Can only be invoked on top-level cluster group.
[23/25] ignite git commit: Merge remote-tracking branch
'origin/master'
Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/master'
Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/fa358cc1
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/fa358cc1
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/fa358cc1
Branch: refs/heads/ignite-4371
Commit: fa358cc1b81310cf5ffe5a3da17a3dad842de33f
Parents: b65802b 10ade28
Author: devozerov <vo...@gridgain.com>
Authored: Fri Dec 9 13:10:43 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Dec 9 13:10:43 2016 +0300
----------------------------------------------------------------------
.../ignite/examples/ExampleNodeStartup.java | 2 +-
.../examples/datagrid/CachePutGetExample.java | 2 +-
.../scalar/examples/ScalarJvmCloudExample.scala | 2 +-
.../rest/ClientMemcachedProtocolSelfTest.java | 4 +-
.../rest/protocols/tcp/MockNioSession.java | 25 +-
.../apache/ignite/IgniteSystemProperties.java | 3 +
.../cache/store/CacheLoadOnlyStoreAdapter.java | 6 +-
.../configuration/IgniteConfiguration.java | 50 +-
.../internal/GridEventConsumeHandler.java | 2 +-
.../ignite/internal/GridJobContextImpl.java | 4 +-
.../ignite/internal/GridKernalContext.java | 9 +
.../ignite/internal/GridKernalContextImpl.java | 16 +-
.../internal/GridPerformanceSuggestions.java | 2 +-
.../org/apache/ignite/internal/GridTopic.java | 5 +-
.../ignite/internal/IgniteInternalFuture.java | 11 +
.../apache/ignite/internal/IgniteKernal.java | 85 +-
.../org/apache/ignite/internal/IgnitionEx.java | 32 +-
.../GridClientConnectionManagerAdapter.java | 1 +
.../client/router/impl/GridTcpRouterImpl.java | 1 +
.../managers/communication/GridIoManager.java | 207 ++-
.../managers/communication/GridIoMessage.java | 13 +
.../communication/GridIoMessageFactory.java | 12 +-
.../communication/IgniteIoTestMessage.java | 235 +++
.../processors/cache/GridCacheAdapter.java | 26 +-
.../processors/cache/GridCacheMessage.java | 7 +
.../processors/cache/GridCacheUtils.java | 35 +
.../processors/cache/IgniteCacheProxy.java | 8 +
.../distributed/GridDistributedLockRequest.java | 5 +
.../GridDistributedTxFinishResponse.java | 6 +
.../GridDistributedUnlockRequest.java | 5 +
.../distributed/dht/GridDhtCacheAdapter.java | 3 +-
.../distributed/dht/GridDhtLockResponse.java | 9 +-
.../dht/atomic/GridDhtAtomicCache.java | 5 +-
.../GridDhtAtomicSingleUpdateRequest.java | 5 +
.../dht/atomic/GridDhtAtomicUpdateRequest.java | 5 +
.../atomic/GridNearAtomicFullUpdateRequest.java | 5 +
.../GridNearAtomicSingleUpdateRequest.java | 5 +
.../distributed/near/GridNearGetRequest.java | 5 +
.../local/atomic/GridLocalAtomicCache.java | 3 +
.../query/GridCacheDistributedQueryManager.java | 2 +-
.../cache/query/GridCacheQueryRequest.java | 6 +-
.../transactions/IgniteTxLocalAdapter.java | 8 +-
.../datastreamer/DataStreamProcessor.java | 22 +-
.../internal/processors/igfs/IgfsContext.java | 4 +-
.../processors/igfs/IgfsDataManager.java | 6 +-
.../internal/processors/igfs/IgfsImpl.java | 2 +-
.../internal/processors/odbc/OdbcProcessor.java | 1 +
.../platform/compute/PlatformCompute.java | 6 +
.../tcp/GridTcpMemcachedNioListener.java | 15 +-
.../protocols/tcp/GridTcpRestNioListener.java | 2 +-
.../rest/protocols/tcp/GridTcpRestProtocol.java | 1 +
.../service/GridServiceProcessor.java | 6 +-
.../ignite/internal/util/IgniteUtils.java | 62 +-
.../ignite/internal/util/StripedExecutor.java | 667 +++++++++
.../util/future/GridFinishedFuture.java | 24 +
.../internal/util/future/GridFutureAdapter.java | 15 +-
.../util/future/GridFutureChainListener.java | 30 +-
.../internal/util/ipc/IpcToNioAdapter.java | 2 +-
.../nio/GridAbstractCommunicationClient.java | 12 +-
.../util/nio/GridCommunicationClient.java | 9 +-
.../nio/GridConnectionBytesVerifyFilter.java | 15 +-
.../util/nio/GridNioAsyncNotifyFilter.java | 10 +-
.../internal/util/nio/GridNioCodecFilter.java | 17 +-
.../ignite/internal/util/nio/GridNioFilter.java | 16 +-
.../internal/util/nio/GridNioFilterAdapter.java | 10 +-
.../internal/util/nio/GridNioFilterChain.java | 14 +-
.../ignite/internal/util/nio/GridNioFuture.java | 4 +-
.../util/nio/GridNioRecoveryDescriptor.java | 124 +-
.../ignite/internal/util/nio/GridNioServer.java | 1404 +++++++++++++++---
.../internal/util/nio/GridNioSession.java | 25 +-
.../internal/util/nio/GridNioSessionImpl.java | 65 +-
.../ignite/internal/util/nio/GridNioWorker.java | 48 +
.../util/nio/GridSelectorNioSessionImpl.java | 221 ++-
.../util/nio/GridShmemCommunicationClient.java | 7 +-
.../util/nio/GridTcpNioCommunicationClient.java | 55 +-
.../internal/util/nio/SessionWriteRequest.java | 85 ++
.../internal/util/nio/ssl/GridNioSslFilter.java | 10 +-
.../util/nio/ssl/GridNioSslHandler.java | 4 +-
.../util/tostring/GridToStringBuilder.java | 2 +-
.../communication/tcp/TcpCommunicationSpi.java | 1340 ++++++++++++-----
.../tcp/TcpCommunicationSpiMBean.java | 40 +
.../ignite/spi/discovery/tcp/ServerImpl.java | 14 +-
.../ignite/stream/socket/SocketStreamer.java | 1 +
.../ignite/thread/IgniteThreadFactory.java | 8 +-
.../IgniteSlowClientDetectionSelfTest.java | 1 +
...unicationBalanceMultipleConnectionsTest.java | 28 +
.../IgniteCommunicationBalanceTest.java | 339 +++++
.../communication/IgniteIoTestMessagesTest.java | 95 ++
.../IgniteVariousConnectionNumberTest.java | 166 +++
.../cache/CrossCacheTxRandomOperationsTest.java | 30 +-
...idAbstractCacheInterceptorRebalanceTest.java | 4 +-
...CacheOffHeapMultiThreadedUpdateSelfTest.java | 6 +-
...eAtomicMessageRecovery10ConnectionsTest.java | 28 +
...cMessageRecoveryNoPairedConnectionsTest.java | 47 +
...acheConnectionRecovery10ConnectionsTest.java | 35 +
.../distributed/IgniteCacheCreatePutTest.java | 2 +-
.../IgniteCacheMessageRecoveryAbstractTest.java | 24 +-
.../IgniteCacheMessageWriteTimeoutTest.java | 17 +-
.../dht/IgniteCacheMultiTxLockSelfTest.java | 6 +-
...erNoStripedPoolMultiNodeFullApiSelfTest.java | 35 +
...edNoStripedPoolMultiNodeFullApiSelfTest.java | 35 +
.../TxDeadlockDetectionNoHangsTest.java | 2 +-
.../TxOptimisticDeadlockDetectionTest.java | 29 +-
.../GridServiceProcessorProxySelfTest.java | 2 +-
.../util/future/GridFutureAdapterSelfTest.java | 122 +-
.../nio/impl/GridNioFilterChainSelfTest.java | 32 +-
.../loadtests/nio/GridNioBenchmarkClient.java | 4 +-
.../p2p/GridP2PRecursionTaskSelfTest.java | 2 +-
.../spi/GridTcpSpiForwardingSelfTest.java | 18 +-
.../GridTcpCommunicationSpiAbstractTest.java | 28 +-
...mmunicationSpiConcurrentConnectSelfTest.java | 82 +-
.../GridTcpCommunicationSpiConfigSelfTest.java | 5 +-
...cpCommunicationSpiMultithreadedSelfTest.java | 23 +-
...dTcpCommunicationSpiRecoveryAckSelfTest.java | 9 +-
...tionSpiRecoveryFailureDetectionSelfTest.java | 1 +
...ationSpiRecoveryNoPairedConnectionsTest.java | 28 +
...GridTcpCommunicationSpiRecoverySelfTest.java | 67 +-
...CommunicationRecoveryAckClosureSelfTest.java | 9 +-
.../junits/GridTestKernalContext.java | 4 +-
.../IgniteCacheFullApiSelfTestSuite.java | 6 +
.../ignite/testsuites/IgniteCacheTestSuite.java | 17 +-
.../IgniteSpiCommunicationSelfTestSuite.java | 2 +
.../hadoop/jobtracker/HadoopJobTracker.java | 4 +-
.../HadoopExternalCommunication.java | 5 +-
.../communication/HadoopIpcToNioAdapter.java | 2 +-
.../communication/HadoopMarshallerFilter.java | 6 +-
.../ignite/stream/kafka/KafkaStreamer.java | 2 +-
.../ignite/tools/classgen/ClassesGenerator.java | 8 +-
.../ignite/yardstick/IgniteBenchmarkUtils.java | 6 +-
.../yardstick/cache/CacheEntryEventProbe.java | 2 +-
.../yardstick/cache/IgniteIoTestBenchmark.java | 73 +
.../io/IgniteIoTestAbstractBenchmark.java | 61 +
.../io/IgniteIoTestSendAllBenchmark.java | 32 +
.../io/IgniteIoTestSendRandomBenchmark.java | 35 +
134 files changed, 5935 insertions(+), 998 deletions(-)
----------------------------------------------------------------------
[04/25] ignite git commit: Improved exception handling for failed
queries.
Posted by sb...@apache.org.
Improved exception handling for failed queries.
Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/0d4a1b73
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/0d4a1b73
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/0d4a1b73
Branch: refs/heads/ignite-4371
Commit: 0d4a1b7381fece47ee480f3a06bff7c51a7fead4
Parents: 6ba1711
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Wed Dec 7 18:02:49 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Wed Dec 7 18:02:49 2016 +0700
----------------------------------------------------------------------
.../java/org/apache/ignite/internal/visor/query/VisorQueryJob.java | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/0d4a1b73/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java
index 6d1de6a..c66b2dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java
@@ -197,7 +197,7 @@ public class VisorQueryJob extends VisorJob<VisorQueryArg, IgniteBiTuple<? exten
}
}
}
- catch (Exception e) {
+ catch (Throwable e) {
return new IgniteBiTuple<>(new VisorExceptionWrapper(e), null);
}
}
[03/25] ignite git commit: IGNITE-4242 ExchangeManager should wait
for cache rebalancing in async way
Posted by sb...@apache.org.
IGNITE-4242 ExchangeManager should wait for cache rebalancing in async way
Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/6ba1711a
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/6ba1711a
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/6ba1711a
Branch: refs/heads/ignite-4371
Commit: 6ba1711a1fa10d8276974227491136070c3ed43a
Parents: acf20b3
Author: Anton Vinogradov <av...@apache.org>
Authored: Tue Dec 6 12:55:41 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Tue Dec 6 12:55:41 2016 +0300
----------------------------------------------------------------------
.../GridCachePartitionExchangeManager.java | 128 +++--------
.../processors/cache/GridCachePreloader.java | 11 +-
.../cache/GridCachePreloaderAdapter.java | 5 +-
.../dht/preloader/GridDhtPartitionDemander.java | 230 +++++++++++--------
.../dht/preloader/GridDhtPreloader.java | 9 +-
.../GridCacheRebalancingSyncSelfTest.java | 2 +
6 files changed, 183 insertions(+), 202 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/6ba1711a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 7a24aa1..f04a6ce 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -21,18 +21,18 @@ import java.text.DateFormat;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Collection;
+import java.util.Collections;
import java.util.Comparator;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
+import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
-import java.util.Queue;
import java.util.TreeMap;
import java.util.UUID;
import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.LinkedBlockingDeque;
@@ -87,7 +87,6 @@ import org.apache.ignite.internal.util.typedef.CI2;
import org.apache.ignite.internal.util.typedef.F;
import org.apache.ignite.internal.util.typedef.T2;
import org.apache.ignite.internal.util.typedef.internal.CU;
-import org.apache.ignite.internal.util.typedef.internal.GPC;
import org.apache.ignite.internal.util.typedef.internal.S;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.internal.util.worker.GridWorker;
@@ -97,13 +96,11 @@ import org.apache.ignite.lang.IgniteUuid;
import org.apache.ignite.thread.IgniteThread;
import org.jetbrains.annotations.Nullable;
import org.jsr166.ConcurrentHashMap8;
-import org.jsr166.ConcurrentLinkedDeque8;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static org.apache.ignite.IgniteSystemProperties.IGNITE_PRELOAD_RESEND_TIMEOUT;
import static org.apache.ignite.IgniteSystemProperties.IGNITE_THREAD_DUMP_ON_EXCHANGE_TIMEOUT;
import static org.apache.ignite.IgniteSystemProperties.getLong;
-import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_STARTED;
import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
@@ -156,9 +153,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
/** */
private GridFutureAdapter<?> reconnectExchangeFut;
- /** */
- private final Queue<Callable<Boolean>> rebalanceQ = new ConcurrentLinkedDeque8<>();
-
/**
* Partition map futures.
* This set also contains already completed exchange futures to address race conditions when coordinator
@@ -1596,12 +1590,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
@Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
long timeout = cctx.gridConfig().getNetworkTimeout();
- boolean startEvtFired = false;
-
int cnt = 0;
- IgniteInternalFuture asyncStartFut = null;
-
while (!isCancelled()) {
GridDhtPartitionsExchangeFuture exchFut = null;
@@ -1703,20 +1693,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
continue;
changed |= cacheCtx.topology().afterExchange(exchFut);
-
- // Preload event notification.
- if (!exchFut.skipPreload() && cacheCtx.events().isRecordable(EVT_CACHE_REBALANCE_STARTED)) {
- if (!cacheCtx.isReplicated() || !startEvtFired) {
- DiscoveryEvent discoEvt = exchFut.discoveryEvent();
-
- cacheCtx.events().addPreloadEvent(-1, EVT_CACHE_REBALANCE_STARTED,
- discoEvt.eventNode(), discoEvt.type(), discoEvt.timestamp());
- }
- }
}
- startEvtFired = true;
-
if (!cctx.kernalContext().clientNode() && changed && futQ.isEmpty())
refreshPartitions();
}
@@ -1755,8 +1733,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
if (assignsMap != null) {
int size = assignsMap.size();
- rebalanceQ.clear();
-
NavigableMap<Integer, List<Integer>> orderMap = new TreeMap<>();
for (Map.Entry<Integer, GridDhtPreloaderAssignments> e : assignsMap.entrySet()) {
@@ -1772,101 +1748,65 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
orderMap.get(order).add(cacheId);
}
- Callable<Boolean> marshR = null;
- List<Callable<Boolean>> orderedRs = new ArrayList<>(size);
+ Runnable r = null;
+
+ List<String> rebList = new LinkedList<>();
+
+ boolean assignsCancelled = false;
- //Ordered rebalance scheduling.
- for (Integer order : orderMap.keySet()) {
+ for (Integer order : orderMap.descendingKeySet()) {
for (Integer cacheId : orderMap.get(order)) {
GridCacheContext<K, V> cacheCtx = cctx.cacheContext(cacheId);
- List<String> waitList = new ArrayList<>(size - 1);
+ GridDhtPreloaderAssignments assigns = assignsMap.get(cacheId);
- for (List<Integer> cIds : orderMap.headMap(order).values()) {
- for (Integer cId : cIds)
- waitList.add(cctx.cacheContext(cId).name());
- }
+ if (assigns != null)
+ assignsCancelled |= assigns.cancelled();
- Callable<Boolean> r = cacheCtx.preloader().addAssignments(assignsMap.get(cacheId),
+ // Cancels previous rebalance future (in case it's not done yet).
+ // Sends previous rebalance stopped event (if necessary).
+ // Creates new rebalance future.
+ // Sends current rebalance started event (if necessary).
+ // Finishes cache sync future (on empty assignments).
+ Runnable cur = cacheCtx.preloader().addAssignments(assigns,
forcePreload,
- waitList,
- cnt);
+ cnt,
+ r);
- if (r != null) {
- U.log(log, "Cache rebalancing scheduled: [cache=" + cacheCtx.name() +
- ", waitList=" + waitList.toString() + "]");
+ if (cur != null) {
+ rebList.add(U.maskName(cacheCtx.name()));
- if (cacheId == CU.cacheId(GridCacheUtils.MARSH_CACHE_NAME))
- marshR = r;
- else
- orderedRs.add(r);
+ r = cur;
}
}
}
- if (asyncStartFut != null)
- asyncStartFut.get(); // Wait for thread stop.
+ if (assignsCancelled) { // Pending exchange.
+ U.log(log, "Skipping rebalancing (obsolete exchange ID) " +
+ "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
+ ", node=" + exchFut.discoveryEvent().eventNode().id() + ']');
+ }
+ else if (r != null) {
+ Collections.reverse(rebList);
- rebalanceQ.addAll(orderedRs);
+ U.log(log, "Rebalancing scheduled [order=" + rebList + "]");
- if (marshR != null || !rebalanceQ.isEmpty()) {
if (futQ.isEmpty()) {
- U.log(log, "Rebalancing required " +
+ U.log(log, "Rebalancing started " +
"[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
", node=" + exchFut.discoveryEvent().eventNode().id() + ']');
- if (marshR != null) {
- try {
- marshR.call(); //Marshaller cache rebalancing launches in sync way.
- }
- catch (Exception ex) {
- if (log.isDebugEnabled())
- log.debug("Failed to send initial demand request to node");
-
- continue;
- }
- }
-
- final GridFutureAdapter fut = new GridFutureAdapter();
-
- asyncStartFut = fut;
-
- cctx.kernalContext().closure().callLocalSafe(new GPC<Boolean>() {
- @Override public Boolean call() {
- try {
- while (true) {
- Callable<Boolean> r = rebalanceQ.poll();
-
- if (r == null)
- return false;
-
- if (!r.call())
- return false;
- }
- }
- catch (Exception ex) {
- if (log.isDebugEnabled())
- log.debug("Failed to send initial demand request to node");
-
- return false;
- }
- finally {
- fut.onDone();
- }
- }
- }, /*system pool*/true);
+ r.run(); // Starts rebalancing routine.
}
- else {
+ else
U.log(log, "Skipping rebalancing (obsolete exchange ID) " +
"[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
", node=" + exchFut.discoveryEvent().eventNode().id() + ']');
- }
}
- else {
+ else
U.log(log, "Skipping rebalancing (nothing scheduled) " +
"[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
", node=" + exchFut.discoveryEvent().eventNode().id() + ']');
- }
}
}
catch (IgniteInterruptedCheckedException e) {
http://git-wip-us.apache.org/repos/asf/ignite/blob/6ba1711a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
index 1d1cfab..3c4456d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache;
import java.util.Collection;
import java.util.UUID;
-import java.util.concurrent.Callable;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
@@ -84,14 +83,14 @@ public interface GridCachePreloader {
*
* @param assignments Assignments to add.
* @param forcePreload Force preload flag.
- * @param caches Rebalancing of these caches will be finished before this started.
* @param cnt Counter.
- * @return Rebalancing closure.
+ * @param next Runnable responsible for cache rebalancing start.
+ * @return Rebalancing runnable.
*/
- public Callable<Boolean> addAssignments(GridDhtPreloaderAssignments assignments,
+ public Runnable addAssignments(GridDhtPreloaderAssignments assignments,
boolean forcePreload,
- Collection<String> caches,
- int cnt);
+ int cnt,
+ Runnable next);
/**
* @param p Preload predicate.
http://git-wip-us.apache.org/repos/asf/ignite/blob/6ba1711a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
index b15ebc5..656a960 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache;
import java.util.Collection;
import java.util.UUID;
-import java.util.concurrent.Callable;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteLogger;
import org.apache.ignite.cache.affinity.AffinityFunction;
@@ -166,8 +165,8 @@ public class GridCachePreloaderAdapter implements GridCachePreloader {
}
/** {@inheritDoc} */
- @Override public Callable<Boolean> addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload,
- Collection<String> caches, int cnt) {
+ @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload,
+ int cnt, Runnable next) {
return null;
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/6ba1711a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index 57d5229..a6808c7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -28,8 +28,8 @@ import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Callable;
import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.ReadWriteLock;
@@ -72,6 +72,7 @@ import org.jetbrains.annotations.Nullable;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_OBJECT_LOADED;
import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_LOADED;
+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.GridTopic.TOPIC_CACHE;
import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.MOVING;
@@ -122,6 +123,18 @@ public class GridDhtPartitionDemander {
private final Map<Integer, Object> rebalanceTopics;
/**
+ * Started event sent.
+ * Make sense for replicated cache only.
+ */
+ private final AtomicBoolean startedEvtSent = new AtomicBoolean();
+
+ /**
+ * Stopped event sent.
+ * Make sense for replicated cache only.
+ */
+ private final AtomicBoolean stoppedEvtSent = new AtomicBoolean();
+
+ /**
* @param cctx Cctx.
* @param demandLock Demand lock.
*/
@@ -249,45 +262,25 @@ public class GridDhtPartitionDemander {
}
/**
- * @param name Cache name.
- * @param fut Future.
- * @throws IgniteCheckedException If failed.
+ * Sets last exchange future.
+ *
+ * @param lastFut Last future to set.
*/
- private boolean waitForCacheRebalancing(String name, RebalanceFuture fut) throws IgniteCheckedException {
- if (log.isDebugEnabled())
- log.debug("Waiting for another cache to start rebalancing [cacheName=" + cctx.name() +
- ", waitCache=" + name + ']');
-
- RebalanceFuture wFut = (RebalanceFuture)cctx.kernalContext().cache().internalCache(name)
- .preloader().rebalanceFuture();
-
- if (!topologyChanged(fut) && wFut.updateSeq == fut.updateSeq) {
- if (!wFut.get()) {
- U.log(log, "Skipping waiting of " + name + " cache [top=" + fut.topologyVersion() +
- "] (cache rebalanced with missed partitions)");
-
- return false;
- }
-
- return true;
- }
- else {
- U.log(log, "Skipping waiting of " + name + " cache [top=" + fut.topologyVersion() +
- "] (topology already changed)");
-
- return false;
- }
+ void onTopologyChanged(GridDhtPartitionsExchangeFuture lastFut) {
+ lastExchangeFut = lastFut;
}
/**
* @param assigns Assignments.
* @param force {@code True} if dummy reassign.
- * @param caches Rebalancing of these caches will be finished before this started.
* @param cnt Counter.
- * @return Rebalancing closure.
+ * @param next Runnable responsible for cache rebalancing start.
+ * @return Rebalancing runnable.
*/
- Callable<Boolean> addAssignments(final GridDhtPreloaderAssignments assigns, boolean force,
- final Collection<String> caches, int cnt) {
+ Runnable addAssignments(final GridDhtPreloaderAssignments assigns,
+ boolean force,
+ int cnt,
+ final Runnable next) {
if (log.isDebugEnabled())
log.debug("Adding partition assignments: " + assigns);
@@ -296,7 +289,7 @@ public class GridDhtPartitionDemander {
if (delay == 0 || force) {
final RebalanceFuture oldFut = rebalanceFut;
- final RebalanceFuture fut = new RebalanceFuture(assigns, cctx, log, oldFut.isInitial(), cnt);
+ final RebalanceFuture fut = new RebalanceFuture(assigns, cctx, log, startedEvtSent, stoppedEvtSent, cnt);
if (!oldFut.isInitial())
oldFut.cancel();
@@ -310,20 +303,69 @@ public class GridDhtPartitionDemander {
rebalanceFut = fut;
- if (assigns.isEmpty()) {
- fut.doneIfEmpty(assigns.cancelled());
+ fut.sendRebalanceStartedEvent();
+
+ if (assigns.cancelled()) { // Pending exchange.
+ if (log.isDebugEnabled())
+ log.debug("Rebalancing skipped due to cancelled assignments.");
+
+ fut.onDone(false);
+
+ fut.sendRebalanceFinishedEvent();
+
+ return null;
+ }
+
+ if (assigns.isEmpty()) { // Nothing to rebalance.
+ if (log.isDebugEnabled())
+ log.debug("Rebalancing skipped due to empty assignments.");
+
+ fut.onDone(true);
+
+ ((GridFutureAdapter)cctx.preloader().syncFuture()).onDone();
+
+ fut.sendRebalanceFinishedEvent();
return null;
}
- return new Callable<Boolean>() {
- @Override public Boolean call() throws Exception {
- for (String c : caches) {
- if (!waitForCacheRebalancing(c, fut))
- return false;
+ return new Runnable() {
+ @Override public void run() {
+ try {
+ if (next != null)
+ fut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
+ @Override public void apply(IgniteInternalFuture<Boolean> f) {
+ try {
+ if (f.get()) // Not cancelled.
+ next.run(); // Starts next cache rebalancing (according to the order).
+ }
+ catch (IgniteCheckedException ignored) {
+ if (log.isDebugEnabled())
+ log.debug(ignored.getMessage());
+ }
+ }
+ });
+
+ requestPartitions(fut, assigns);
}
+ catch (IgniteCheckedException e) {
+ ClusterTopologyCheckedException cause = e.getCause(ClusterTopologyCheckedException.class);
- return requestPartitions(fut, assigns);
+ if (cause != null)
+ log.warning("Failed to send initial demand request to node. " + e.getMessage());
+ else
+ log.error("Failed to send initial demand request to node.", e);
+
+ fut.cancel();
+ }
+ catch (Throwable th) {
+ log.error("Runtime error caught during initial demand request sending.", th);
+
+ fut.cancel();
+
+ if (th instanceof Error)
+ throw th;
+ }
}
};
}
@@ -361,14 +403,17 @@ public class GridDhtPartitionDemander {
* @throws IgniteCheckedException If failed.
* @return Partitions were requested.
*/
- private boolean requestPartitions(
+ private void requestPartitions(
RebalanceFuture fut,
GridDhtPreloaderAssignments assigns
) throws IgniteCheckedException {
- for (Map.Entry<ClusterNode, GridDhtPartitionDemandMessage> e : assigns.entrySet()) {
- if (topologyChanged(fut))
- return false;
+ if (topologyChanged(fut)) {
+ fut.cancel();
+
+ return;
+ }
+ for (Map.Entry<ClusterNode, GridDhtPartitionDemandMessage> e : assigns.entrySet()) {
final ClusterNode node = e.getKey();
GridDhtPartitionDemandMessage d = e.getValue();
@@ -387,7 +432,7 @@ public class GridDhtPartitionDemander {
//Check remote node rebalancing API version.
if (node.version().compareTo(GridDhtPreloader.REBALANCING_VER_2_SINCE) >= 0) {
- U.log(log, "Starting rebalancing [cache=" + cctx.name() + ", mode=" + cfg.getRebalanceMode() +
+ U.log(log, "Starting rebalancing [mode=" + cfg.getRebalanceMode() +
", fromNode=" + node.id() + ", partitionsCount=" + parts.size() +
", topology=" + fut.topologyVersion() + ", updateSeq=" + fut.updateSeq + "]");
@@ -446,8 +491,6 @@ public class GridDhtPartitionDemander {
worker.run(node, d);
}
}
-
- return true;
}
/**
@@ -739,23 +782,17 @@ public class GridDhtPartitionDemander {
}
/**
- * Sets last exchange future.
- *
- * @param lastFut Last future to set.
- */
- void updateLastExchangeFuture(GridDhtPartitionsExchangeFuture lastFut) {
- lastExchangeFut = lastFut;
- }
-
- /**
*
*/
public static class RebalanceFuture extends GridFutureAdapter<Boolean> {
/** */
private static final long serialVersionUID = 1L;
- /** Should EVT_CACHE_REBALANCE_STOPPED event be sent of not. */
- private final boolean sndStoppedEvnt;
+ /** Should EVT_CACHE_REBALANCE_STARTED event be sent or not. */
+ private final AtomicBoolean startedEvtSent;
+
+ /** Should EVT_CACHE_REBALANCE_STOPPED event be sent or not. */
+ private final AtomicBoolean stoppedEvtSent;
/** */
private final GridCacheContext<?, ?> cctx;
@@ -783,13 +820,15 @@ public class GridDhtPartitionDemander {
* @param assigns Assigns.
* @param cctx Context.
* @param log Logger.
- * @param sentStopEvnt Stop event flag.
+ * @param startedEvtSent Start event sent flag.
+ * @param stoppedEvtSent Stop event sent flag.
* @param updateSeq Update sequence.
*/
RebalanceFuture(GridDhtPreloaderAssignments assigns,
GridCacheContext<?, ?> cctx,
IgniteLogger log,
- boolean sentStopEvnt,
+ AtomicBoolean startedEvtSent,
+ AtomicBoolean stoppedEvtSent,
long updateSeq) {
assert assigns != null;
@@ -797,7 +836,8 @@ public class GridDhtPartitionDemander {
this.topVer = assigns.topologyVersion();
this.cctx = cctx;
this.log = log;
- this.sndStoppedEvnt = sentStopEvnt;
+ this.startedEvtSent = startedEvtSent;
+ this.stoppedEvtSent = stoppedEvtSent;
this.updateSeq = updateSeq;
}
@@ -809,7 +849,8 @@ public class GridDhtPartitionDemander {
this.topVer = null;
this.cctx = null;
this.log = null;
- this.sndStoppedEvnt = false;
+ this.startedEvtSent = null;
+ this.stoppedEvtSent = null;
this.updateSeq = -1;
}
@@ -848,24 +889,6 @@ public class GridDhtPartitionDemander {
}
/**
- * @param cancelled Is cancelled.
- */
- private void doneIfEmpty(boolean cancelled) {
- synchronized (this) {
- if (isDone())
- return;
-
- assert remaining.isEmpty();
-
- if (log.isDebugEnabled())
- log.debug("Rebalancing is not required [cache=" + cctx.name() +
- ", topology=" + topVer + "]");
-
- checkIsDone(cancelled, true);
- }
- }
-
- /**
* Cancels this future.
*
* @return {@code True}.
@@ -875,8 +898,7 @@ public class GridDhtPartitionDemander {
if (isDone())
return true;
- U.log(log, "Cancelled rebalancing from all nodes [cache=" + cctx.name()
- + ", topology=" + topologyVersion() + ']');
+ U.log(log, "Cancelled rebalancing from all nodes [topology=" + topologyVersion() + ']');
if (!cctx.kernalContext().isStopping()) {
for (UUID nodeId : remaining.keySet())
@@ -885,7 +907,7 @@ public class GridDhtPartitionDemander {
remaining.clear();
- checkIsDone(true /* cancelled */, false);
+ checkIsDone(true /* cancelled */);
}
return true;
@@ -907,7 +929,7 @@ public class GridDhtPartitionDemander {
remaining.remove(nodeId);
- onDone(false); // Finishing rebalance future a non completed.
+ onDone(false); // Finishing rebalance future as non completed.
checkIsDone(); // But will finish syncFuture only when other nodes are preloaded or rebalancing cancelled.
}
@@ -988,8 +1010,7 @@ public class GridDhtPartitionDemander {
if (parts.isEmpty()) {
U.log(log, "Completed " + ((remaining.size() == 1 ? "(final) " : "") +
- "rebalancing [cache=" + cctx.name() +
- ", fromNode=" + nodeId + ", topology=" + topologyVersion() +
+ "rebalancing [fromNode=" + nodeId + ", topology=" + topologyVersion() +
", time=" + (U.currentTimeMillis() - t.get1()) + " ms]"));
remaining.remove(nodeId);
@@ -1022,23 +1043,20 @@ public class GridDhtPartitionDemander {
*
*/
private void checkIsDone() {
- checkIsDone(false, false);
+ checkIsDone(false);
}
/**
* @param cancelled Is cancelled.
- * @param wasEmpty {@code True} if future was created without assignments.
*/
- private void checkIsDone(boolean cancelled, boolean wasEmpty) {
+ private void checkIsDone(boolean cancelled) {
if (remaining.isEmpty()) {
- if (cctx.events().isRecordable(EVT_CACHE_REBALANCE_STOPPED) && (!cctx.isReplicated() || sndStoppedEvnt))
- preloadEvent(EVT_CACHE_REBALANCE_STOPPED, exchFut.discoveryEvent());
+ sendRebalanceFinishedEvent();
if (log.isDebugEnabled())
log.debug("Completed rebalance future: " + this);
- if (!wasEmpty)
- cctx.shared().exchange().scheduleResendPartitions();
+ cctx.shared().exchange().scheduleResendPartitions();
Collection<Integer> m = new HashSet<>();
@@ -1064,6 +1082,30 @@ public class GridDhtPartitionDemander {
}
}
+ /**
+ *
+ */
+ private void sendRebalanceStartedEvent() {
+ if (cctx.events().isRecordable(EVT_CACHE_REBALANCE_STARTED) &&
+ (!cctx.isReplicated() || !startedEvtSent.get())) {
+ preloadEvent(EVT_CACHE_REBALANCE_STARTED, exchFut.discoveryEvent());
+
+ startedEvtSent.set(true);
+ }
+ }
+
+ /**
+ *
+ */
+ private void sendRebalanceFinishedEvent() {
+ if (cctx.events().isRecordable(EVT_CACHE_REBALANCE_STOPPED) &&
+ (!cctx.isReplicated() || !stoppedEvtSent.get())) {
+ preloadEvent(EVT_CACHE_REBALANCE_STOPPED, exchFut.discoveryEvent());
+
+ stoppedEvtSent.set(true);
+ }
+ }
+
/** {@inheritDoc} */
public String toString() {
return S.toString(RebalanceFuture.class, this);
http://git-wip-us.apache.org/repos/asf/ignite/blob/6ba1711a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
index 0865d9f..692e7c0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
@@ -22,7 +22,6 @@ import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.UUID;
-import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.ReadWriteLock;
@@ -255,7 +254,7 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
@Override public void onTopologyChanged(GridDhtPartitionsExchangeFuture lastFut) {
supplier.onTopologyChanged(lastFut.topologyVersion());
- demander.updateLastExchangeFuture(lastFut);
+ demander.onTopologyChanged(lastFut);
}
/** {@inheritDoc} */
@@ -413,9 +412,9 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
}
/** {@inheritDoc} */
- @Override public Callable<Boolean> addAssignments(GridDhtPreloaderAssignments assignments,
- boolean forcePreload, Collection<String> caches, int cnt) {
- return demander.addAssignments(assignments, forcePreload, caches, cnt);
+ @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments,
+ boolean forcePreload, int cnt, Runnable next) {
+ return demander.addAssignments(assignments, forcePreload, cnt, next);
}
/**
http://git-wip-us.apache.org/repos/asf/ignite/blob/6ba1711a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
index de38952..3dfcd85 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
@@ -501,6 +501,8 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
record = true;
+ log.info("Checking GridDhtPartitions*Message absent (it will take 30 SECONDS) ... ");
+
U.sleep(30_000);
record = false;
[13/25] ignite git commit: Merge remote-tracking branch
'origin/master'
Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/master'
Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/2f51b4ac
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/2f51b4ac
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/2f51b4ac
Branch: refs/heads/ignite-4371
Commit: 2f51b4ac0966d6786f4a172023e2f238ac20aba0
Parents: 9a691c4 c14561c
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Dec 8 18:04:11 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Dec 8 18:04:11 2016 +0300
----------------------------------------------------------------------
----------------------------------------------------------------------
[08/25] ignite git commit: IGNITE-4367 .NET: Fix ServicesTest
Posted by sb...@apache.org.
IGNITE-4367 .NET: Fix ServicesTest
Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/597f3a58
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/597f3a58
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/597f3a58
Branch: refs/heads/ignite-4371
Commit: 597f3a581a0cc5e6061fd49071c13856be9d3627
Parents: 858b129
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Thu Dec 8 13:18:31 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Thu Dec 8 13:18:31 2016 +0300
----------------------------------------------------------------------
.../Apache.Ignite.Core.Tests/Services/ServicesTest.cs | 9 +++------
1 file changed, 3 insertions(+), 6 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/597f3a58/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
index 38a96bd..dcddfdb 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
@@ -762,9 +762,6 @@ namespace Apache.Ignite.Core.Tests.Services
[InstanceResource]
private IIgnite _grid;
- /** */
- private readonly object _syncRoot = new object();
-
/** <inheritdoc /> */
public int TestProperty { get; set; }
@@ -810,7 +807,7 @@ namespace Apache.Ignite.Core.Tests.Services
/** <inheritdoc /> */
public void Init(IServiceContext context)
{
- lock (_syncRoot)
+ lock (this)
{
if (ThrowInit)
throw new Exception("Expected exception");
@@ -825,7 +822,7 @@ namespace Apache.Ignite.Core.Tests.Services
/** <inheritdoc /> */
public void Execute(IServiceContext context)
{
- lock (_syncRoot)
+ lock (this)
{
if (ThrowExecute)
throw new Exception("Expected exception");
@@ -843,7 +840,7 @@ namespace Apache.Ignite.Core.Tests.Services
/** <inheritdoc /> */
public void Cancel(IServiceContext context)
{
- lock (_syncRoot)
+ lock (this)
{
if (ThrowCancel)
throw new Exception("Expected exception");
[14/25] ignite git commit: IGNITE-4270: Hadoop: implemented striped
mapper output. This closes #1334.
Posted by sb...@apache.org.
IGNITE-4270: Hadoop: implemented striped mapper output. This closes #1334.
Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/065ca4a7
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/065ca4a7
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/065ca4a7
Branch: refs/heads/ignite-4371
Commit: 065ca4a75a0765409a27d87c781efb215c0a6c48
Parents: 2f51b4a
Author: devozerov <vo...@gridgain.com>
Authored: Fri Dec 9 12:01:40 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Dec 9 12:01:40 2016 +0300
----------------------------------------------------------------------
.../communication/GridIoMessageFactory.java | 6 +
.../processors/hadoop/HadoopJobProperty.java | 7 +
.../hadoop/HadoopMapperAwareTaskOutput.java | 32 ++
.../processors/hadoop/HadoopTaskInfo.java | 43 ++
.../shuffle/HadoopDirectShuffleMessage.java | 243 ++++++++++++
.../processors/hadoop/HadoopMapperUtils.java | 56 +++
.../hadoop/impl/v2/HadoopV2Context.java | 11 +
.../hadoop/impl/v2/HadoopV2MapTask.java | 10 +
.../hadoop/jobtracker/HadoopJobTracker.java | 4 +
.../hadoop/shuffle/HadoopShuffle.java | 23 +-
.../hadoop/shuffle/HadoopShuffleJob.java | 389 ++++++++++++++-----
.../shuffle/HadoopShuffleRemoteState.java | 5 +-
.../shuffle/direct/HadoopDirectDataInput.java | 166 ++++++++
.../shuffle/direct/HadoopDirectDataOutput.java | 221 +++++++++++
.../direct/HadoopDirectDataOutputContext.java | 100 +++++
.../direct/HadoopDirectDataOutputState.java | 54 +++
.../child/HadoopChildProcessRunner.java | 2 +-
.../impl/HadoopMapReduceEmbeddedSelfTest.java | 22 +-
18 files changed, 1287 insertions(+), 107 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index 4ffb220..504e683 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@ -122,6 +122,7 @@ import org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleAck;
import org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleFinishRequest;
import org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleFinishResponse;
import org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleMessage;
+import org.apache.ignite.internal.processors.hadoop.shuffle.HadoopDirectShuffleMessage;
import org.apache.ignite.internal.processors.igfs.IgfsAckMessage;
import org.apache.ignite.internal.processors.igfs.IgfsBlockKey;
import org.apache.ignite.internal.processors.igfs.IgfsBlocksMessage;
@@ -170,6 +171,11 @@ public class GridIoMessageFactory implements MessageFactory {
Message msg = null;
switch (type) {
+ case -42:
+ msg = new HadoopDirectShuffleMessage();
+
+ break;
+
case -41:
msg = new HadoopShuffleFinishResponse();
http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
index e713caa..1f0ef1b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
@@ -91,6 +91,13 @@ public enum HadoopJobProperty {
SHUFFLE_MSG_SIZE("ignite.shuffle.message.size"),
/**
+ * Whether to stripe mapper output for remote reducers.
+ * <p>
+ * Defaults to {@code false}.
+ */
+ SHUFFLE_MAPPER_STRIPED_OUTPUT("ignite.shuffle.mapper.striped.output"),
+
+ /**
* Shuffle job throttle in milliseconds. When job is executed with separate shuffle thread, this parameter
* controls sleep duration between iterations through intermediate reducer maps.
* <p>
http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapperAwareTaskOutput.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapperAwareTaskOutput.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapperAwareTaskOutput.java
new file mode 100644
index 0000000..1d6637c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapperAwareTaskOutput.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.hadoop;
+
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * Special output type with callback invoked when mapper finished writing data.
+ */
+public interface HadoopMapperAwareTaskOutput extends HadoopTaskOutput {
+ /**
+ * Callback invoked when mapper finished writing data.
+ *
+ * @throws IgniteCheckedException If failed.
+ */
+ public void onMapperFinished() throws IgniteCheckedException;
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskInfo.java
index b76fb85..3509367 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskInfo.java
@@ -46,6 +46,12 @@ public class HadoopTaskInfo implements Externalizable {
/** */
private HadoopInputSplit inputSplit;
+ /** Whether mapper index is set. */
+ private boolean mapperIdxSet;
+
+ /** Current mapper index. */
+ private int mapperIdx;
+
/**
* For {@link Externalizable}.
*/
@@ -78,6 +84,13 @@ public class HadoopTaskInfo implements Externalizable {
out.writeInt(taskNum);
out.writeInt(attempt);
out.writeObject(inputSplit);
+
+ if (mapperIdxSet) {
+ out.writeBoolean(true);
+ out.writeInt(mapperIdx);
+ }
+ else
+ out.writeBoolean(false);
}
/** {@inheritDoc} */
@@ -87,6 +100,13 @@ public class HadoopTaskInfo implements Externalizable {
taskNum = in.readInt();
attempt = in.readInt();
inputSplit = (HadoopInputSplit)in.readObject();
+
+ if (in.readBoolean()) {
+ mapperIdxSet = true;
+ mapperIdx = in.readInt();
+ }
+ else
+ mapperIdxSet = false;
}
/**
@@ -118,6 +138,29 @@ public class HadoopTaskInfo implements Externalizable {
}
/**
+ * @param mapperIdx Current mapper index.
+ */
+ public void mapperIndex(int mapperIdx) {
+ this.mapperIdx = mapperIdx;
+
+ mapperIdxSet = true;
+ }
+
+ /**
+ * @return Current mapper index or {@code null}
+ */
+ public int mapperIndex() {
+ return mapperIdx;
+ }
+
+ /**
+ * @return {@code True} if mapped index is set.
+ */
+ public boolean hasMapperIndex() {
+ return mapperIdxSet;
+ }
+
+ /**
* @return Input split.
*/
@Nullable public HadoopInputSplit inputSplit() {
http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopDirectShuffleMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopDirectShuffleMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopDirectShuffleMessage.java
new file mode 100644
index 0000000..e81dc5f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopDirectShuffleMessage.java
@@ -0,0 +1,243 @@
+/*
+ * 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.hadoop.shuffle;
+
+import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
+import org.apache.ignite.internal.processors.hadoop.message.HadoopMessage;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.nio.ByteBuffer;
+
+/**
+ * Direct shuffle message.
+ */
+public class HadoopDirectShuffleMessage implements Message, HadoopMessage {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ @GridToStringInclude
+ private HadoopJobId jobId;
+
+ /** */
+ @GridToStringInclude
+ private int reducer;
+
+ /** Count. */
+ private int cnt;
+
+ /** Buffer. */
+ private byte[] buf;
+
+ /** Buffer length (equal or less than buf.length). */
+ @GridDirectTransient
+ private transient int bufLen;
+
+ /**
+ * Default constructor.
+ */
+ public HadoopDirectShuffleMessage() {
+ // No-op.
+ }
+
+ /**
+ * Constructor.
+ *
+ * @param jobId Job ID.
+ * @param reducer Reducer.
+ * @param cnt Count.
+ * @param buf Buffer.
+ * @param bufLen Buffer length.
+ */
+ public HadoopDirectShuffleMessage(HadoopJobId jobId, int reducer, int cnt, byte[] buf, int bufLen) {
+ assert jobId != null;
+
+ this.jobId = jobId;
+ this.reducer = reducer;
+ this.cnt = cnt;
+ this.buf = buf;
+ this.bufLen = bufLen;
+ }
+
+ /**
+ * @return Job ID.
+ */
+ public HadoopJobId jobId() {
+ return jobId;
+ }
+
+ /**
+ * @return Reducer.
+ */
+ public int reducer() {
+ return reducer;
+ }
+
+ /**
+ * @return Count.
+ */
+ public int count() {
+ return cnt;
+ }
+
+ /**
+ * @return Buffer.
+ */
+ public byte[] buffer() {
+ return buf;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+ writer.setBuffer(buf);
+
+ if (!writer.isHeaderWritten()) {
+ if (!writer.writeHeader(directType(), fieldsCount()))
+ return false;
+
+ writer.onHeaderWritten();
+ }
+
+ switch (writer.state()) {
+ case 0:
+ if (!writer.writeMessage("jobId", jobId))
+ return false;
+
+ writer.incrementState();
+
+ case 1:
+ if (!writer.writeInt("reducer", reducer))
+ return false;
+
+ writer.incrementState();
+
+ case 2:
+ if (!writer.writeInt("cnt", cnt))
+ return false;
+
+ writer.incrementState();
+
+ case 3:
+ if (!writer.writeByteArray("buf", this.buf, 0, bufLen))
+ return false;
+
+ writer.incrementState();
+
+ }
+
+ return true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+ reader.setBuffer(buf);
+
+ if (!reader.beforeMessageRead())
+ return false;
+
+ switch (reader.state()) {
+ case 0:
+ jobId = reader.readMessage("jobId");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 1:
+ reducer = reader.readInt("reducer");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 2:
+ cnt = reader.readInt("cnt");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 3:
+ this.buf = reader.readByteArray("buf");
+
+ if (!reader.isLastRead())
+ return false;
+
+ bufLen = this.buf != null ? this.buf.length : 0;
+
+ reader.incrementState();
+
+ }
+
+ return reader.afterMessageRead(HadoopDirectShuffleMessage.class);
+ }
+
+ /** {@inheritDoc} */
+ @Override public byte directType() {
+ return -42;
+ }
+
+ /** {@inheritDoc} */
+ @Override public byte fieldsCount() {
+ return 4;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onAckReceived() {
+ // No-op.
+ }
+
+ /** {@inheritDoc} */
+ @Override public void writeExternal(ObjectOutput out) throws IOException {
+ jobId.writeExternal(out);
+
+ out.writeInt(reducer);
+ out.writeInt(cnt);
+
+ U.writeByteArray(out, buf);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+ jobId = new HadoopJobId();
+ jobId.readExternal(in);
+
+ reducer = in.readInt();
+ cnt = in.readInt();
+
+ buf = U.readByteArray(in);
+ bufLen = buf != null ? buf.length : 0;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(HadoopDirectShuffleMessage.class, this);
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapperUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapperUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapperUtils.java
new file mode 100644
index 0000000..87adcb7
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapperUtils.java
@@ -0,0 +1,56 @@
+/*
+ * 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.hadoop;
+
+/**
+ * Set of mapper utility methods.
+ */
+public class HadoopMapperUtils {
+ /** Thread-local mapper index. */
+ private static final ThreadLocal<Integer> MAP_IDX = new ThreadLocal<>();
+
+ /**
+ * @return Current mapper index.
+ */
+ public static int mapperIndex() {
+ Integer res = MAP_IDX.get();
+
+ return res != null ? res : -1;
+ }
+
+ /**
+ * @param idx Current mapper index.
+ */
+ public static void mapperIndex(Integer idx) {
+ MAP_IDX.set(idx);
+ }
+
+ /**
+ * Clear mapper index.
+ */
+ public static void clearMapperIndex() {
+ MAP_IDX.remove();
+ }
+
+ /**
+ * Constructor.
+ */
+ private HadoopMapperUtils() {
+ // No-op.
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Context.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Context.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Context.java
index 90a1bad..eec0636 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Context.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Context.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.mapreduce.task.JobContextImpl;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.internal.processors.hadoop.HadoopFileBlock;
import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
+import org.apache.ignite.internal.processors.hadoop.HadoopMapperAwareTaskOutput;
import org.apache.ignite.internal.processors.hadoop.HadoopTaskCancelledException;
import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
import org.apache.ignite.internal.processors.hadoop.HadoopTaskInput;
@@ -153,6 +154,16 @@ public class HadoopV2Context extends JobContextImpl implements MapContext, Reduc
}
}
+ /**
+ * Callback invoked from mapper thread when map is finished.
+ *
+ * @throws IgniteCheckedException If failed.
+ */
+ public void onMapperFinished() throws IgniteCheckedException {
+ if (output instanceof HadoopMapperAwareTaskOutput)
+ ((HadoopMapperAwareTaskOutput)output).onMapperFinished();
+ }
+
/** {@inheritDoc} */
@Override public OutputCommitter getOutputCommitter() {
throw new UnsupportedOperationException();
http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2MapTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2MapTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2MapTask.java
index 418df4e..eb3b935 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2MapTask.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2MapTask.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.util.ReflectionUtils;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.internal.IgniteInterruptedCheckedException;
import org.apache.ignite.internal.processors.hadoop.HadoopJobInfo;
+import org.apache.ignite.internal.processors.hadoop.HadoopMapperUtils;
import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo;
/**
@@ -49,6 +50,11 @@ public class HadoopV2MapTask extends HadoopV2Task {
JobContextImpl jobCtx = taskCtx.jobContext();
+ if (taskCtx.taskInfo().hasMapperIndex())
+ HadoopMapperUtils.mapperIndex(taskCtx.taskInfo().mapperIndex());
+ else
+ HadoopMapperUtils.clearMapperIndex();
+
try {
InputSplit nativeSplit = hadoopContext().getInputSplit();
@@ -72,6 +78,8 @@ public class HadoopV2MapTask extends HadoopV2Task {
try {
mapper.run(new WrappedMapper().getMapContext(hadoopContext()));
+
+ hadoopContext().onMapperFinished();
}
finally {
closeWriter();
@@ -92,6 +100,8 @@ public class HadoopV2MapTask extends HadoopV2Task {
throw new IgniteCheckedException(e);
}
finally {
+ HadoopMapperUtils.clearMapperIndex();
+
if (err != null)
abort(outputFormat);
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
index 36782bf..a725534 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
@@ -1018,6 +1018,8 @@ public class HadoopJobTracker extends HadoopComponent {
if (state == null)
state = initState(jobId);
+ int mapperIdx = 0;
+
for (HadoopInputSplit split : mappers) {
if (state.addMapper(split)) {
if (log.isDebugEnabled())
@@ -1026,6 +1028,8 @@ public class HadoopJobTracker extends HadoopComponent {
HadoopTaskInfo taskInfo = new HadoopTaskInfo(MAP, jobId, meta.taskNumber(split), 0, split);
+ taskInfo.mapperIndex(mapperIdx++);
+
if (tasks == null)
tasks = new ArrayList<>();
http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
index 82bbd32..8ffea8c 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.managers.communication.GridIoPolicy;
import org.apache.ignite.internal.managers.communication.GridMessageListener;
import org.apache.ignite.internal.processors.hadoop.HadoopComponent;
import org.apache.ignite.internal.processors.hadoop.HadoopContext;
+import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan;
import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
@@ -39,6 +40,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.lang.IgniteBiPredicate;
import org.apache.ignite.plugin.extensions.communication.Message;
+import java.util.Collection;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
@@ -102,8 +104,8 @@ public class HadoopShuffle extends HadoopComponent {
private HadoopShuffleJob<UUID> newJob(HadoopJobId jobId) throws IgniteCheckedException {
HadoopMapReducePlan plan = ctx.jobTracker().plan(jobId);
- HadoopShuffleJob<UUID> job = new HadoopShuffleJob<>(ctx.localNodeId(), log,
- ctx.jobTracker().job(jobId, null), mem, plan.reducers(), plan.reducers(ctx.localNodeId()), true);
+ HadoopShuffleJob<UUID> job = new HadoopShuffleJob<>(ctx.localNodeId(), log, ctx.jobTracker().job(jobId, null),
+ mem, plan.reducers(), plan.reducers(ctx.localNodeId()), localMappersCount(plan), true);
UUID[] rdcAddrs = new UUID[plan.reducers()];
@@ -123,6 +125,18 @@ public class HadoopShuffle extends HadoopComponent {
}
/**
+ * Get number of local mappers.
+ *
+ * @param plan Plan.
+ * @return Number of local mappers.
+ */
+ private int localMappersCount(HadoopMapReducePlan plan) {
+ Collection<HadoopInputSplit> locMappers = plan.mappers(ctx.localNodeId());
+
+ return F.isEmpty(locMappers) ? 0 : locMappers.size();
+ }
+
+ /**
* @param nodeId Node ID to send message to.
* @param msg Message to send.
* @throws IgniteCheckedException If send failed.
@@ -195,6 +209,11 @@ public class HadoopShuffle extends HadoopComponent {
job(m.jobId()).onShuffleMessage(src, m);
}
+ else if (msg instanceof HadoopDirectShuffleMessage) {
+ HadoopDirectShuffleMessage m = (HadoopDirectShuffleMessage)msg;
+
+ job(m.jobId()).onDirectShuffleMessage(src, m);
+ }
else if (msg instanceof HadoopShuffleAck) {
HadoopShuffleAck m = (HadoopShuffleAck)msg;
http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
index 0a3a0ae..214a335 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
@@ -17,20 +17,16 @@
package org.apache.ignite.internal.processors.hadoop.shuffle;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicReferenceArray;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteLogger;
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.IgniteInterruptedCheckedException;
import org.apache.ignite.internal.processors.hadoop.HadoopJob;
import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
+import org.apache.ignite.internal.processors.hadoop.HadoopMapperAwareTaskOutput;
+import org.apache.ignite.internal.processors.hadoop.HadoopMapperUtils;
import org.apache.ignite.internal.processors.hadoop.HadoopPartitioner;
+import org.apache.ignite.internal.processors.hadoop.HadoopSerialization;
import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo;
import org.apache.ignite.internal.processors.hadoop.HadoopTaskInput;
@@ -41,6 +37,9 @@ import org.apache.ignite.internal.processors.hadoop.message.HadoopMessage;
import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopConcurrentHashMultimap;
import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopMultimap;
import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopSkipList;
+import org.apache.ignite.internal.processors.hadoop.shuffle.direct.HadoopDirectDataInput;
+import org.apache.ignite.internal.processors.hadoop.shuffle.direct.HadoopDirectDataOutputContext;
+import org.apache.ignite.internal.processors.hadoop.shuffle.direct.HadoopDirectDataOutputState;
import org.apache.ignite.internal.util.GridUnsafe;
import org.apache.ignite.internal.util.future.GridCompoundFuture;
import org.apache.ignite.internal.util.future.GridFinishedFuture;
@@ -55,9 +54,19 @@ import org.apache.ignite.internal.util.worker.GridWorker;
import org.apache.ignite.lang.IgniteBiTuple;
import org.apache.ignite.lang.IgniteInClosure;
import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicReferenceArray;
import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.PARTITION_HASHMAP_SIZE;
import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.SHUFFLE_JOB_THROTTLE;
+import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.SHUFFLE_MAPPER_STRIPED_OUTPUT;
import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.SHUFFLE_MSG_SIZE;
import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.SHUFFLE_REDUCER_NO_SORTING;
import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.get;
@@ -121,6 +130,9 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
/** Message size. */
private final int msgSize;
+ /** Whether to strip mappers for remote execution. */
+ private final boolean stripeMappers;
+
/** Local shuffle states. */
private volatile HashMap<T, HadoopShuffleLocalState> locShuffleStates = new HashMap<>();
@@ -143,11 +155,12 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
* @param mem Memory.
* @param totalReducerCnt Amount of reducers in the Job.
* @param locReducers Reducers will work on current node.
+ * @param locMappersCnt Number of mappers running on the given node.
* @param embedded Whether shuffle is running in embedded mode.
* @throws IgniteCheckedException If error.
*/
public HadoopShuffleJob(T locReduceAddr, IgniteLogger log, HadoopJob job, GridUnsafeMemory mem,
- int totalReducerCnt, int[] locReducers, boolean embedded) throws IgniteCheckedException {
+ int totalReducerCnt, int[] locReducers, int locMappersCnt, boolean embedded) throws IgniteCheckedException {
this.locReduceAddr = locReduceAddr;
this.totalReducerCnt = totalReducerCnt;
this.job = job;
@@ -155,6 +168,27 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
this.log = log.getLogger(HadoopShuffleJob.class);
this.embedded = embedded;
+ // No stripes for combiner.
+ boolean stripeMappers0 = get(job.info(), SHUFFLE_MAPPER_STRIPED_OUTPUT, false);
+
+ if (stripeMappers0) {
+ if (job.info().hasCombiner()) {
+ log.info("Striped mapper output is disabled because it cannot be used together with combiner [jobId=" +
+ job.id() + ']');
+
+ stripeMappers0 = false;
+ }
+
+ if (!embedded) {
+ log.info("Striped mapper output is disabled becuase it cannot be used in external mode [jobId=" +
+ job.id() + ']');
+
+ stripeMappers0 = false;
+ }
+ }
+
+ stripeMappers = stripeMappers0;
+
msgSize = get(job.info(), SHUFFLE_MSG_SIZE, DFLT_SHUFFLE_MSG_SIZE);
locReducersCtx = new AtomicReferenceArray<>(totalReducerCnt);
@@ -169,9 +203,20 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
needPartitioner = totalReducerCnt > 1;
+ // Size of local map is always equal to total reducer number to allow index-based lookup.
locMaps = new AtomicReferenceArray<>(totalReducerCnt);
- rmtMaps = new AtomicReferenceArray<>(totalReducerCnt);
- msgs = new HadoopShuffleMessage[totalReducerCnt];
+
+ // Size of remote map:
+ // - If there are no local mappers, then we will not send anything, so set to 0;
+ // - If output is not striped, then match it to total reducer count, the same way as for local maps.
+ // - If output is striped, then multiply previous value by number of local mappers.
+ int rmtMapsSize = locMappersCnt == 0 ? 0 : totalReducerCnt;
+
+ if (stripeMappers)
+ rmtMapsSize *= locMappersCnt;
+
+ rmtMaps = new AtomicReferenceArray<>(rmtMapsSize);
+ msgs = new HadoopShuffleMessage[rmtMapsSize];
throttle = get(job.info(), SHUFFLE_JOB_THROTTLE, 0);
}
@@ -208,24 +253,26 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
this.io = io;
- if (!flushed) {
- snd = new GridWorker(gridName, "hadoop-shuffle-" + job.id(), log) {
- @Override protected void body() throws InterruptedException {
- try {
- while (!isCancelled()) {
- if (throttle > 0)
- Thread.sleep(throttle);
-
- collectUpdatesAndSend(false);
+ if (!stripeMappers) {
+ if (!flushed) {
+ snd = new GridWorker(gridName, "hadoop-shuffle-" + job.id(), log) {
+ @Override protected void body() throws InterruptedException {
+ try {
+ while (!isCancelled()) {
+ if (throttle > 0)
+ Thread.sleep(throttle);
+
+ collectUpdatesAndSend(false);
+ }
+ }
+ catch (IgniteCheckedException e) {
+ throw new IllegalStateException(e);
}
}
- catch (IgniteCheckedException e) {
- throw new IllegalStateException(e);
- }
- }
- };
+ };
- new IgniteThread(snd).start();
+ new IgniteThread(snd).start();
+ }
}
ioInitLatch.countDown();
@@ -306,6 +353,46 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
}
/**
+ * Process shuffle message.
+ *
+ * @param src Source.
+ * @param msg Message.
+ * @throws IgniteCheckedException Exception.
+ */
+ public void onDirectShuffleMessage(T src, HadoopDirectShuffleMessage msg) throws IgniteCheckedException {
+ assert msg.buffer() != null;
+
+ HadoopTaskContext taskCtx = locReducersCtx.get(msg.reducer()).get();
+
+ HadoopPerformanceCounter perfCntr = HadoopPerformanceCounter.getCounter(taskCtx.counters(), null);
+
+ perfCntr.onShuffleMessage(msg.reducer(), U.currentTimeMillis());
+
+ HadoopMultimap map = getOrCreateMap(locMaps, msg.reducer());
+
+ HadoopSerialization keySer = taskCtx.keySerialization();
+ HadoopSerialization valSer = taskCtx.valueSerialization();
+
+ // Add data from message to the map.
+ try (HadoopMultimap.Adder adder = map.startAdding(taskCtx)) {
+ HadoopDirectDataInput in = new HadoopDirectDataInput(msg.buffer());
+
+ Object key = null;
+ Object val = null;
+
+ for (int i = 0; i < msg.count(); i++) {
+ key = keySer.read(in, key);
+ val = valSer.read(in, val);
+
+ adder.write(key, val);
+ }
+ }
+
+ if (localShuffleState(src).onShuffleMessage())
+ sendFinishResponse(src, msg.jobId());
+ }
+
+ /**
* @param ack Shuffle ack.
*/
@SuppressWarnings("ConstantConditions")
@@ -467,88 +554,149 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
}
/**
- * Sends map updates to remote reducers.
+ * Send updates to remote reducers.
+ *
+ * @param flush Flush flag.
+ * @throws IgniteCheckedException If failed.
*/
private void collectUpdatesAndSend(boolean flush) throws IgniteCheckedException {
- for (int i = 0; i < rmtMaps.length(); i++) {
- HadoopMultimap map = rmtMaps.get(i);
+ for (int i = 0; i < rmtMaps.length(); i++)
+ collectUpdatesAndSend(i, flush);
+ }
+
+ /**
+ * Send updates to concrete remote reducer.
+ *
+ * @param rmtMapIdx Remote map index.
+ * @param flush Flush flag.
+ * @throws IgniteCheckedException If failed.
+ */
+ private void collectUpdatesAndSend(int rmtMapIdx, boolean flush) throws IgniteCheckedException {
+ final int rmtRdcIdx = stripeMappers ? rmtMapIdx % totalReducerCnt : rmtMapIdx;
- if (map == null)
- continue; // Skip empty map and local node.
+ HadoopMultimap map = rmtMaps.get(rmtMapIdx);
- if (msgs[i] == null)
- msgs[i] = new HadoopShuffleMessage(job.id(), i, msgSize);
+ if (map == null)
+ return;
- final int idx = i;
+ if (msgs[rmtMapIdx] == null)
+ msgs[rmtMapIdx] = new HadoopShuffleMessage(job.id(), rmtRdcIdx, msgSize);
- map.visit(false, new HadoopMultimap.Visitor() {
- /** */
- private long keyPtr;
+ visit(map, rmtMapIdx, rmtRdcIdx);
- /** */
- private int keySize;
+ if (flush && msgs[rmtMapIdx].offset() != 0)
+ send(rmtMapIdx, rmtRdcIdx, 0);
+ }
- /** */
- private boolean keyAdded;
+ /**
+ * Flush remote direct context.
+ *
+ * @param rmtMapIdx Remote map index.
+ * @param rmtDirectCtx Remote direct context.
+ * @param reset Whether to perform reset.
+ */
+ private void sendShuffleMessage(int rmtMapIdx, @Nullable HadoopDirectDataOutputContext rmtDirectCtx, boolean reset) {
+ if (rmtDirectCtx == null)
+ return;
- /** {@inheritDoc} */
- @Override public void onKey(long keyPtr, int keySize) {
- this.keyPtr = keyPtr;
- this.keySize = keySize;
+ int cnt = rmtDirectCtx.count();
- keyAdded = false;
- }
+ if (cnt == 0)
+ return;
- private boolean tryAdd(long valPtr, int valSize) {
- HadoopShuffleMessage msg = msgs[idx];
+ int rmtRdcIdx = stripeMappers ? rmtMapIdx % totalReducerCnt : rmtMapIdx;
- if (!keyAdded) { // Add key and value.
- int size = keySize + valSize;
+ HadoopDirectDataOutputState state = rmtDirectCtx.state();
- if (!msg.available(size, false))
- return false;
+ if (reset)
+ rmtDirectCtx.reset();
- msg.addKey(keyPtr, keySize);
- msg.addValue(valPtr, valSize);
+ HadoopDirectShuffleMessage msg = new HadoopDirectShuffleMessage(job.id(), rmtRdcIdx, cnt,
+ state.buffer(), state.bufferLength());
- keyAdded = true;
+ T nodeId = reduceAddrs[rmtRdcIdx];
- return true;
- }
+ io.apply(nodeId, msg);
+
+ remoteShuffleState(nodeId).onShuffleMessage();
+ }
+
+ /**
+ * Visit output map.
+ *
+ * @param map Map.
+ * @param rmtMapIdx Remote map index.
+ * @param rmtRdcIdx Remote reducer index.
+ * @throws IgniteCheckedException If failed.
+ */
+ private void visit(HadoopMultimap map, final int rmtMapIdx, final int rmtRdcIdx) throws IgniteCheckedException {
+ map.visit(false, new HadoopMultimap.Visitor() {
+ /** */
+ private long keyPtr;
- if (!msg.available(valSize, true))
+ /** */
+ private int keySize;
+
+ /** */
+ private boolean keyAdded;
+
+ /** {@inheritDoc} */
+ @Override public void onKey(long keyPtr, int keySize) {
+ this.keyPtr = keyPtr;
+ this.keySize = keySize;
+
+ keyAdded = false;
+ }
+
+ private boolean tryAdd(long valPtr, int valSize) {
+ HadoopShuffleMessage msg = msgs[rmtMapIdx];
+
+ if (!keyAdded) { // Add key and value.
+ int size = keySize + valSize;
+
+ if (!msg.available(size, false))
return false;
+ msg.addKey(keyPtr, keySize);
msg.addValue(valPtr, valSize);
+ keyAdded = true;
+
return true;
}
- /** {@inheritDoc} */
- @Override public void onValue(long valPtr, int valSize) {
- if (tryAdd(valPtr, valSize))
- return;
+ if (!msg.available(valSize, true))
+ return false;
- send(idx, keySize + valSize);
+ msg.addValue(valPtr, valSize);
- keyAdded = false;
+ return true;
+ }
- if (!tryAdd(valPtr, valSize))
- throw new IllegalStateException();
- }
- });
+ /** {@inheritDoc} */
+ @Override public void onValue(long valPtr, int valSize) {
+ if (tryAdd(valPtr, valSize))
+ return;
- if (flush && msgs[i].offset() != 0)
- send(i, 0);
- }
+ send(rmtMapIdx, rmtRdcIdx, keySize + valSize);
+
+ keyAdded = false;
+
+ if (!tryAdd(valPtr, valSize))
+ throw new IllegalStateException();
+ }
+ });
}
/**
- * @param idx Index of message.
+ * Send message.
+ *
+ * @param rmtMapIdx Remote map index.
+ * @param rmtRdcIdx Remote reducer index.
* @param newBufMinSize Min new buffer size.
*/
- private void send(final int idx, int newBufMinSize) {
- HadoopShuffleMessage msg = msgs[idx];
+ private void send(int rmtMapIdx, int rmtRdcIdx, int newBufMinSize) {
+ HadoopShuffleMessage msg = msgs[rmtMapIdx];
final long msgId = msg.id();
@@ -566,10 +714,10 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
}
try {
- io.apply(reduceAddrs[idx], msg);
+ io.apply(reduceAddrs[rmtRdcIdx], msg);
if (embedded)
- remoteShuffleState(reduceAddrs[idx]).onShuffleMessage();
+ remoteShuffleState(reduceAddrs[rmtRdcIdx]).onShuffleMessage();
}
catch (GridClosureException e) {
if (fut != null)
@@ -593,7 +741,7 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
});
}
- msgs[idx] = newBufMinSize == 0 ? null : new HadoopShuffleMessage(job.id(), idx,
+ msgs[rmtMapIdx] = newBufMinSize == 0 ? null : new HadoopShuffleMessage(job.id(), rmtRdcIdx,
Math.max(msgSize, newBufMinSize));
}
@@ -639,31 +787,33 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
if (totalReducerCnt == 0)
return new GridFinishedFuture<>();
- U.await(ioInitLatch);
+ if (!stripeMappers) {
+ U.await(ioInitLatch);
- GridWorker snd0 = snd;
+ GridWorker snd0 = snd;
- if (snd0 != null) {
- if (log.isDebugEnabled())
- log.debug("Cancelling sender thread.");
+ if (snd0 != null) {
+ if (log.isDebugEnabled())
+ log.debug("Cancelling sender thread.");
- snd0.cancel();
+ snd0.cancel();
- try {
- snd0.join();
+ try {
+ snd0.join();
- if (log.isDebugEnabled())
- log.debug("Finished waiting for sending thread to complete on shuffle job flush: " + job.id());
- }
- catch (InterruptedException e) {
- throw new IgniteInterruptedCheckedException(e);
+ if (log.isDebugEnabled())
+ log.debug("Finished waiting for sending thread to complete on shuffle job flush: " + job.id());
+ }
+ catch (InterruptedException e) {
+ throw new IgniteInterruptedCheckedException(e);
+ }
}
- }
- collectUpdatesAndSend(true); // With flush.
+ collectUpdatesAndSend(true); // With flush.
- if (log.isDebugEnabled())
- log.debug("Finished sending collected updates to remote reducers: " + job.id());
+ if (log.isDebugEnabled())
+ log.debug("Finished sending collected updates to remote reducers: " + job.id());
+ }
GridCompoundFuture fut = new GridCompoundFuture<>();
@@ -700,8 +850,8 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
if (log.isDebugEnabled())
log.debug("Collected futures to compound futures for flush: " + sentMsgs.size());
-
}
+
return fut;
}
@@ -775,13 +925,17 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
/**
* Partitioned output.
*/
- private class PartitionedOutput implements HadoopTaskOutput {
+ public class PartitionedOutput implements HadoopMapperAwareTaskOutput {
/** */
private final HadoopTaskOutput[] locAdders = new HadoopTaskOutput[locMaps.length()];
/** */
private final HadoopTaskOutput[] rmtAdders = new HadoopTaskOutput[rmtMaps.length()];
+ /** Remote direct contexts. */
+ private final HadoopDirectDataOutputContext[] rmtDirectCtxs =
+ new HadoopDirectDataOutputContext[rmtMaps.length()];
+
/** */
private HadoopPartitioner partitioner;
@@ -819,16 +973,53 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
locAdders[part] = out = getOrCreateMap(locMaps, part).startAdding(taskCtx);
}
else {
- out = rmtAdders[part];
+ if (stripeMappers) {
+ int mapperIdx = HadoopMapperUtils.mapperIndex();
- if (out == null)
- rmtAdders[part] = out = getOrCreateMap(rmtMaps, part).startAdding(taskCtx);
+ assert mapperIdx >= 0;
+
+ int idx = totalReducerCnt * mapperIdx + part;
+
+ HadoopDirectDataOutputContext rmtDirectCtx = rmtDirectCtxs[idx];
+
+ if (rmtDirectCtx == null) {
+ rmtDirectCtx = new HadoopDirectDataOutputContext(msgSize, taskCtx);
+
+ rmtDirectCtxs[idx] = rmtDirectCtx;
+ }
+
+ if (rmtDirectCtx.write(key, val))
+ sendShuffleMessage(idx, rmtDirectCtx, true);
+
+ return;
+ }
+ else {
+ out = rmtAdders[part];
+
+ if (out == null)
+ rmtAdders[part] = out = getOrCreateMap(rmtMaps, part).startAdding(taskCtx);
+ }
}
out.write(key, val);
}
/** {@inheritDoc} */
+ @Override public void onMapperFinished() throws IgniteCheckedException {
+ if (stripeMappers) {
+ int mapperIdx = HadoopMapperUtils.mapperIndex();
+
+ assert mapperIdx >= 0;
+
+ for (int i = 0; i < totalReducerCnt; i++) {
+ int idx = totalReducerCnt * mapperIdx + i;
+
+ sendShuffleMessage(idx, rmtDirectCtxs[idx], false);
+ }
+ }
+ }
+
+ /** {@inheritDoc} */
@Override public void close() throws IgniteCheckedException {
for (HadoopTaskOutput adder : locAdders) {
if (adder != null)
http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleRemoteState.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleRemoteState.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleRemoteState.java
index 5ffaa55..4331124 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleRemoteState.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleRemoteState.java
@@ -17,17 +17,14 @@
package org.apache.ignite.internal.processors.hadoop.shuffle;
-import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.util.future.GridFutureAdapter;
-import org.apache.ignite.lang.IgniteInClosure;
-import java.util.UUID;
import java.util.concurrent.atomic.AtomicLong;
/**
* Remote shuffle state.
*/
-class HadoopShuffleRemoteState<T> {
+class HadoopShuffleRemoteState {
/** Message count. */
private final AtomicLong msgCnt = new AtomicLong();
http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataInput.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataInput.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataInput.java
new file mode 100644
index 0000000..e3a713a
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataInput.java
@@ -0,0 +1,166 @@
+/*
+ * 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.hadoop.shuffle.direct;
+
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.jetbrains.annotations.NotNull;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+
+import static org.apache.ignite.internal.util.GridUnsafe.BYTE_ARR_OFF;
+
+/**
+ * Hadoop data input used for direct communication.
+ */
+public class HadoopDirectDataInput extends InputStream implements DataInput {
+ /** Data buffer. */
+ private final byte[] buf;
+
+ /** Position. */
+ private int pos;
+
+ /**
+ * Constructor.
+ *
+ * @param buf Buffer.
+ */
+ public HadoopDirectDataInput(byte[] buf) {
+ this.buf = buf;
+ }
+
+ /** {@inheritDoc} */
+ @Override public int read() throws IOException {
+ return readByte();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void readFully(@NotNull byte[] b) throws IOException {
+ readFully(b, 0, b.length);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void readFully(@NotNull byte[] b, int off, int len) throws IOException {
+ System.arraycopy(buf, pos, b, off, len);
+
+ pos += len;
+ }
+
+ /** {@inheritDoc} */
+ @Override public int skipBytes(int n) throws IOException {
+ pos += n;
+
+ return n;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean readBoolean() throws IOException {
+ return readByte() == 1;
+ }
+
+ /** {@inheritDoc} */
+ @Override public byte readByte() throws IOException {
+ byte res = GridUnsafe.getByte(buf, BYTE_ARR_OFF + pos);
+
+ pos += 1;
+
+ return res;
+ }
+
+ /** {@inheritDoc} */
+ @Override public int readUnsignedByte() throws IOException {
+ return readByte() & 0xff;
+ }
+
+ /** {@inheritDoc} */
+ @Override public short readShort() throws IOException {
+ short res = GridUnsafe.getShort(buf, BYTE_ARR_OFF + pos);
+
+ pos += 2;
+
+ return res;
+ }
+
+ /** {@inheritDoc} */
+ @Override public int readUnsignedShort() throws IOException {
+ return readShort() & 0xffff;
+ }
+
+ /** {@inheritDoc} */
+ @Override public char readChar() throws IOException {
+ char res = GridUnsafe.getChar(buf, BYTE_ARR_OFF + pos);
+
+ pos += 2;
+
+ return res;
+ }
+
+ /** {@inheritDoc} */
+ @Override public int readInt() throws IOException {
+ int res = GridUnsafe.getInt(buf, BYTE_ARR_OFF + pos);
+
+ pos += 4;
+
+ return res;
+ }
+
+ /** {@inheritDoc} */
+ @Override public long readLong() throws IOException {
+ long res = GridUnsafe.getLong(buf, BYTE_ARR_OFF + pos);
+
+ pos += 8;
+
+ return res;
+ }
+
+ /** {@inheritDoc} */
+ @Override public float readFloat() throws IOException {
+ float res = GridUnsafe.getFloat(buf, BYTE_ARR_OFF + pos);
+
+ pos += 4;
+
+ return res;
+ }
+
+ /** {@inheritDoc} */
+ @Override public double readDouble() throws IOException {
+ double res = GridUnsafe.getDouble(buf, BYTE_ARR_OFF + pos);
+
+ pos += 8;
+
+ return res;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String readLine() throws IOException {
+ // TODO: Create ticket!
+ throw new UnsupportedOperationException();
+ }
+
+ /** {@inheritDoc} */
+ @NotNull @Override public String readUTF() throws IOException {
+ byte[] bytes = new byte[readShort()];
+
+ if (bytes.length != 0)
+ readFully(bytes);
+
+ return new String(bytes, StandardCharsets.UTF_8);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutput.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutput.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutput.java
new file mode 100644
index 0000000..151e552
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutput.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.hadoop.shuffle.direct;
+
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.jetbrains.annotations.NotNull;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.UTFDataFormatException;
+import java.nio.charset.StandardCharsets;
+
+import static org.apache.ignite.internal.util.GridUnsafe.BYTE_ARR_OFF;
+
+/**
+ * Hadoop data output for direct communication.
+ */
+public class HadoopDirectDataOutput extends OutputStream implements DataOutput {
+ /** Flush size. */
+ private final int flushSize;
+
+ /** Data buffer. */
+ private byte[] buf;
+
+ /** Buffer size. */
+ private int bufSize;
+
+ /** Position. */
+ private int pos;
+
+ /**
+ * Constructor.
+ *
+ * @param flushSize Flush size.
+ */
+ public HadoopDirectDataOutput(int flushSize) {
+ this(flushSize, flushSize);
+ }
+
+ /**
+ * Constructor.
+ *
+ * @param flushSize Flush size.
+ * @param allocSize Allocation size.
+ */
+ public HadoopDirectDataOutput(int flushSize, int allocSize) {
+ this.flushSize = flushSize;
+
+ buf = new byte[allocSize];
+ bufSize = allocSize;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void write(@NotNull byte[] b) throws IOException {
+ write(b, 0, b.length);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void write(@NotNull byte[] b, int off, int len) throws IOException {
+ int writePos = ensure(len);
+
+ System.arraycopy(b, off, buf, writePos, len);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void write(int val) throws IOException {
+ writeByte(val);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void writeBoolean(boolean val) throws IOException {
+ writeByte(val ? (byte)1 : (byte)0);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void writeByte(int val) throws IOException {
+ int writePos = ensure(1);
+
+ buf[writePos] = (byte)val;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void writeShort(int val) throws IOException {
+ int writePos = ensure(2);
+
+ GridUnsafe.putShort(buf, BYTE_ARR_OFF + writePos, (short)val);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void writeChar(int val) throws IOException {
+ int writePos = ensure(2);
+
+ GridUnsafe.putChar(buf, BYTE_ARR_OFF + writePos, (char)val);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void writeInt(int val) throws IOException {
+ int writePos = ensure(4);
+
+ GridUnsafe.putInt(buf, BYTE_ARR_OFF + writePos, val);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void writeLong(long val) throws IOException {
+ int writePos = ensure(8);
+
+ GridUnsafe.putLong(buf, BYTE_ARR_OFF + writePos, val);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void writeFloat(float val) throws IOException {
+ int writePos = ensure(4);
+
+ GridUnsafe.putFloat(buf, BYTE_ARR_OFF + writePos, val);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void writeDouble(double val) throws IOException {
+ int writePos = ensure(8);
+
+ GridUnsafe.putDouble(buf, BYTE_ARR_OFF + writePos, val);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void writeBytes(@NotNull String str) throws IOException {
+ for(int i = 0; i < str.length(); ++i)
+ write((byte)str.charAt(i));
+ }
+
+ /** {@inheritDoc} */
+ @Override public void writeChars(@NotNull String str) throws IOException {
+ for (int i = 0; i < str.length(); ++i)
+ writeChar(str.charAt(i));
+ }
+
+ /** {@inheritDoc} */
+ @Override public void writeUTF(@NotNull String str) throws IOException {
+ byte[] bytes = str.getBytes(StandardCharsets.UTF_8);
+
+ int len = bytes.length;
+
+ if (len > 65535)
+ throw new UTFDataFormatException("UTF8 form of string is longer than 65535 bytes: " + str);
+
+ writeShort((short)len);
+ write(bytes);
+ }
+
+ /**
+ * @return Buffer.
+ */
+ public byte[] buffer() {
+ return buf;
+ }
+
+ /**
+ * @return Position.
+ */
+ public int position() {
+ return pos;
+ }
+
+ /**
+ * @return Whether buffer is ready for flush.
+ */
+ public boolean readyForFlush() {
+ return pos >= flushSize;
+ }
+
+ /**
+ * Ensure that the given amount of bytes is available within the stream, then shift the position.
+ *
+ * @param cnt Count.
+ * @return Position
+ */
+ private int ensure(int cnt) {
+ int pos0 = pos;
+
+ if (pos0 + cnt > bufSize)
+ grow(pos0 + cnt);
+
+ pos += cnt;
+
+ return pos0;
+ }
+
+ /**
+ * Grow array up to the given count.
+ *
+ * @param cnt Count.
+ */
+ private void grow(int cnt) {
+ int bufSize0 = (int)(bufSize * 1.1);
+
+ if (bufSize0 < cnt)
+ bufSize0 = cnt;
+
+ byte[] buf0 = new byte[bufSize0];
+
+ System.arraycopy(buf, 0, buf0, 0, pos);
+
+ buf = buf0;
+ bufSize = bufSize0;
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputContext.java
new file mode 100644
index 0000000..bc70ef3
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputContext.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.hadoop.shuffle.direct;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.hadoop.HadoopSerialization;
+import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
+
+/**
+ * Hadoop data output context for direct communication.
+ */
+public class HadoopDirectDataOutputContext {
+ /** Flush size. */
+ private final int flushSize;
+
+ /** Key serialization. */
+ private final HadoopSerialization keySer;
+
+ /** Value serialization. */
+ private final HadoopSerialization valSer;
+
+ /** Data output. */
+ private HadoopDirectDataOutput out;
+
+ /** Number of keys written. */
+ private int cnt;
+
+ /**
+ * Constructor.
+ *
+ * @param flushSize Flush size.
+ * @param taskCtx Task context.
+ * @throws IgniteCheckedException If failed.
+ */
+ public HadoopDirectDataOutputContext(int flushSize, HadoopTaskContext taskCtx)
+ throws IgniteCheckedException {
+ this.flushSize = flushSize;
+
+ keySer = taskCtx.keySerialization();
+ valSer = taskCtx.valueSerialization();
+
+ out = new HadoopDirectDataOutput(flushSize);
+ }
+
+ /**
+ * Write key-value pair.
+ *
+ * @param key Key.
+ * @param val Value.
+ * @return Whether flush is needed.
+ * @throws IgniteCheckedException If failed.
+ */
+ public boolean write(Object key, Object val) throws IgniteCheckedException {
+ keySer.write(out, key);
+ valSer.write(out, val);
+
+ cnt++;
+
+ return out.readyForFlush();
+ }
+
+ /**
+ * @return Key-value pairs count.
+ */
+ public int count() {
+ return cnt;
+ }
+
+ /**
+ * @return State.
+ */
+ public HadoopDirectDataOutputState state() {
+ return new HadoopDirectDataOutputState(out.buffer(), out.position());
+ }
+
+ /**
+ * Reset buffer.
+ */
+ public void reset() {
+ int allocSize = Math.max(flushSize, out.position());
+
+ out = new HadoopDirectDataOutput(flushSize, allocSize);
+ cnt = 0;
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputState.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputState.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputState.java
new file mode 100644
index 0000000..a9c12e3
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputState.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.hadoop.shuffle.direct;
+
+/**
+ * Hadoop data output state for direct communication.
+ */
+public class HadoopDirectDataOutputState {
+ /** Buffer. */
+ private final byte[] buf;
+
+ /** Buffer length. */
+ private final int bufLen;
+
+ /**
+ * Constructor.
+ *
+ * @param buf Buffer.
+ * @param bufLen Buffer length.
+ */
+ public HadoopDirectDataOutputState(byte[] buf, int bufLen) {
+ this.buf = buf;
+ this.bufLen = bufLen;
+ }
+
+ /**
+ * @return Buffer.
+ */
+ public byte[] buffer() {
+ return buf;
+ }
+
+ /**
+ * @return Length.
+ */
+ public int bufferLength() {
+ return bufLen;
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
index cb08c00..3336120 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
@@ -151,7 +151,7 @@ public class HadoopChildProcessRunner {
job.initialize(true, nodeDesc.processId());
shuffleJob = new HadoopShuffleJob<>(comm.localProcessDescriptor(), log, job, mem,
- req.totalReducerCount(), req.localReducers(), false);
+ req.totalReducerCount(), req.localReducers(), 0, false);
initializeExecutors(req);
http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceEmbeddedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceEmbeddedSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceEmbeddedSelfTest.java
index b04deeb..8897a38 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceEmbeddedSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceEmbeddedSelfTest.java
@@ -34,6 +34,7 @@ import org.apache.ignite.configuration.HadoopConfiguration;
import org.apache.ignite.igfs.IgfsPath;
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobProperty;
import org.apache.ignite.internal.processors.hadoop.impl.examples.HadoopWordCount1;
import org.apache.ignite.internal.processors.hadoop.impl.examples.HadoopWordCount2;
@@ -54,12 +55,28 @@ public class HadoopMapReduceEmbeddedSelfTest extends HadoopMapReduceTest {
return cfg;
}
+ /*
+ * @throws Exception If fails.
+ */
+ public void testMultiReducerWholeMapReduceExecution() throws Exception {
+ checkMultiReducerWholeMapReduceExecution(false);
+ }
+
+ /*
+ * @throws Exception If fails.
+ */
+ public void testMultiReducerWholeMapReduceExecutionStriped() throws Exception {
+ checkMultiReducerWholeMapReduceExecution(true);
+ }
+
/**
* Tests whole job execution with all phases in old and new versions of API with definition of custom
* Serialization, Partitioner and IO formats.
+ *
+ * @param striped Whether output should be striped or not.
* @throws Exception If fails.
*/
- public void testMultiReducerWholeMapReduceExecution() throws Exception {
+ public void checkMultiReducerWholeMapReduceExecution(boolean striped) throws Exception {
IgfsPath inDir = new IgfsPath(PATH_INPUT);
igfs.mkdirs(inDir);
@@ -81,6 +98,9 @@ public class HadoopMapReduceEmbeddedSelfTest extends HadoopMapReduceTest {
JobConf jobConf = new JobConf();
+ if (striped)
+ jobConf.set(HadoopJobProperty.SHUFFLE_MAPPER_STRIPED_OUTPUT.propertyName(), "true");
+
jobConf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, CustomSerialization.class.getName());
//To split into about 6-7 items for v2
[16/25] ignite git commit: IGNITE-3220 I/O bottleneck on
server/client cluster configuration Communications optimizations: -
possibility to open separate in/out connections - possibility to have
multiple connections between nodes - implemented NIO sessio
Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
index 482e2ef..c7a1a53 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
@@ -17,6 +17,7 @@
package org.apache.ignite.spi.communication.tcp;
+import org.apache.ignite.internal.util.nio.GridNioServer;
import org.apache.ignite.mxbean.MXBeanDescription;
import org.apache.ignite.spi.IgniteSpiManagementMBean;
@@ -44,6 +45,35 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
public int getLocalPort();
/**
+ * Returns {@code true} if {@code TcpCommunicationSpi} should
+ * maintain connection for outgoing and incoming messages separately.
+ * In this case total number of connections between local and some remote node
+ * is {@link #getConnectionsPerNode()} * 2.
+ * <p>
+ * Returns {@code false} if each connection of {@link #getConnectionsPerNode()}
+ * should be used for outgoing and incoming messages. In this case load NIO selectors load
+ * balancing of {@link GridNioServer} will be disabled.
+ * <p>
+ * Default is {@code true}.
+ *
+ * @return {@code true} to use paired connections and {@code false} otherwise.
+ * @see #getConnectionsPerNode()
+ */
+ @MXBeanDescription("Paired connections used.")
+ public boolean isUsePairedConnections();
+
+ /**
+ * Gets number of connections to each remote node. if {@link #isUsePairedConnections()}
+ * is {@code true} then number of connections is doubled and half is used for incoming and
+ * half for outgoing messages.
+ *
+ * @return Number of connections per node.
+ * @see #isUsePairedConnections()
+ */
+ @MXBeanDescription("Connections per node.")
+ public int getConnectionsPerNode();
+
+ /**
* Gets local port for shared memory communication.
*
* @return Port number.
@@ -153,6 +183,16 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
public int getReconnectCount();
/**
+ * Defines how many non-blocking {@code selector.selectNow()} should be made before
+ * falling into {@code selector.select(long)} in NIO server. Long value. Default is {@code 0}.
+ * Can be set to {@code Long.MAX_VALUE} so selector threads will never block.
+ *
+ * @return Selector thread busy-loop iterations.
+ */
+ @MXBeanDescription("Selector thread busy-loop iterations.")
+ public long getSelectorSpins();
+
+ /**
* Gets value for {@code TCP_NODELAY} socket option.
*
* @return {@code True} if TCP delay is disabled.
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 204b685..50fa3bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -3424,7 +3424,7 @@ class ServerImpl extends TcpDiscoveryImpl {
if (log.isDebugEnabled())
log.debug("Node validation failed [res=" + err + ", node=" + node + ']');
- utilityPool.submit(
+ utilityPool.execute(
new Runnable() {
@Override public void run() {
boolean ping = node.id().equals(err.nodeId()) ? pingNode(node) : pingNode(err.nodeId());
@@ -3469,7 +3469,7 @@ class ServerImpl extends TcpDiscoveryImpl {
final String rmtMarsh = node.attribute(ATTR_MARSHALLER);
if (!F.eq(locMarsh, rmtMarsh)) {
- utilityPool.submit(
+ utilityPool.execute(
new Runnable() {
@Override public void run() {
String errMsg = "Local node's marshaller differs from remote node's marshaller " +
@@ -3526,7 +3526,7 @@ class ServerImpl extends TcpDiscoveryImpl {
boolean locLateAssignBool = locLateAssign != null ? locLateAssign : false;
if (locMarshUseDfltSuidBool != rmtMarshUseDfltSuidBool) {
- utilityPool.submit(
+ utilityPool.execute(
new Runnable() {
@Override public void run() {
String errMsg = "Local node's " + IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID +
@@ -3568,7 +3568,7 @@ class ServerImpl extends TcpDiscoveryImpl {
final boolean rmtMarshCompactFooterBool = rmtMarshCompactFooter != null ? rmtMarshCompactFooter : false;
if (locMarshCompactFooterBool != rmtMarshCompactFooterBool) {
- utilityPool.submit(
+ utilityPool.execute(
new Runnable() {
@Override public void run() {
String errMsg = "Local node's binary marshaller \"compactFooter\" property differs from " +
@@ -3606,7 +3606,7 @@ class ServerImpl extends TcpDiscoveryImpl {
final boolean rmtMarshStrSerialVer2Bool = rmtMarshStrSerialVer2 != null ? rmtMarshStrSerialVer2 : false;
if (locMarshStrSerialVer2Bool != rmtMarshStrSerialVer2Bool) {
- utilityPool.submit(
+ utilityPool.execute(
new Runnable() {
@Override public void run() {
String errMsg = "Local node's " + IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2 +
@@ -3679,7 +3679,7 @@ class ServerImpl extends TcpDiscoveryImpl {
final Boolean rmtSrvcCompatibilityEnabled = node.attribute(ATTR_SERVICES_COMPATIBILITY_MODE);
if (!F.eq(locSrvcCompatibilityEnabled, rmtSrvcCompatibilityEnabled)) {
- utilityPool.submit(
+ utilityPool.execute(
new Runnable() {
@Override public void run() {
String errMsg = "Local node's " + IGNITE_SERVICES_COMPATIBILITY_MODE +
@@ -3714,7 +3714,7 @@ class ServerImpl extends TcpDiscoveryImpl {
}
}
else if (Boolean.FALSE.equals(locSrvcCompatibilityEnabled)) {
- utilityPool.submit(
+ utilityPool.execute(
new Runnable() {
@Override public void run() {
String errMsg = "Remote node doesn't support lazy services configuration and " +
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java b/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java
index d1c8d19..127778b 100644
--- a/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java
+++ b/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java
@@ -184,6 +184,7 @@ public class SocketStreamer<T, K, V> extends StreamAdapter<T, K, V> {
try {
srv = new GridNioServer.Builder<byte[]>()
.address(addr == null ? InetAddress.getLocalHost() : addr)
+ .serverName("sock-streamer")
.port(port)
.listener(lsnr)
.logger(log)
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java
index 55557dd..d173594 100644
--- a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java
@@ -20,6 +20,7 @@ package org.apache.ignite.thread;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.internal.util.typedef.internal.S;
import org.jetbrains.annotations.NotNull;
/**
@@ -62,4 +63,9 @@ public class IgniteThreadFactory implements ThreadFactory {
@Override public Thread newThread(@NotNull Runnable r) {
return new IgniteThread(gridName, threadName, r, idxGen.incrementAndGet());
}
-}
\ No newline at end of file
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(IgniteThreadFactory.class, this, super.toString());
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
index 760313b..5721887 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
@@ -75,6 +75,7 @@ public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
commSpi.setSlowClientQueueLimit(50);
commSpi.setSharedMemoryPort(-1);
commSpi.setIdleConnectionTimeout(300_000);
+ commSpi.setConnectionsPerNode(1);
cfg.setCommunicationSpi(commSpi);
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceMultipleConnectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceMultipleConnectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceMultipleConnectionsTest.java
new file mode 100644
index 0000000..e95b1ec
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceMultipleConnectionsTest.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.managers.communication;
+
+/**
+ *
+ */
+public class IgniteCommunicationBalanceMultipleConnectionsTest extends IgniteCommunicationBalanceTest {
+ /** {@inheritDoc} */
+ @Override protected int connectionsPerNode() {
+ return 5;
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java
new file mode 100644
index 0000000..e142aef
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java
@@ -0,0 +1,339 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.managers.communication;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteCompute;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
+import org.apache.ignite.internal.util.lang.GridAbsPredicateX;
+import org.apache.ignite.internal.util.nio.GridNioServer;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class IgniteCommunicationBalanceTest extends GridCommonAbstractTest {
+ /** */
+ private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+ /** */
+ private boolean client;
+
+ /** */
+ private int selectors;
+
+ /** {@inheritDoc} */
+ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+ IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+ TcpCommunicationSpi commSpi = ((TcpCommunicationSpi)cfg.getCommunicationSpi());
+
+ commSpi.setSharedMemoryPort(-1);
+ commSpi.setConnectionsPerNode(connectionsPerNode());
+
+ if (selectors > 0)
+ commSpi.setSelectorsCount(selectors);
+
+ ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+ cfg.setClientMode(client);
+
+ return cfg;
+ }
+
+ /**
+ * @return Connections per node.
+ */
+ protected int connectionsPerNode() {
+ return 1;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void afterTest() throws Exception {
+ stopAllGrids();
+
+ super.afterTest();
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testBalance1() throws Exception {
+ System.setProperty(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, "5000");
+
+ try {
+ selectors = 4;
+
+ final int SRVS = 4;
+
+ startGridsMultiThreaded(SRVS);
+
+ client = true;
+
+ final Ignite client = startGrid(SRVS);
+
+ for (int i = 0; i < 4; i++) {
+ ClusterNode node = client.cluster().node(ignite(i).cluster().localNode().id());
+
+ client.compute(client.cluster().forNode(node)).call(new DummyCallable(null));
+ }
+
+ waitNioBalanceStop(Collections.singletonList(client), 10_000);
+
+ final GridNioServer srv = GridTestUtils.getFieldValue(client.configuration().getCommunicationSpi(), "nioSrvr");
+
+ ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+ long readMoveCnt1 = srv.readerMoveCount();
+ long writeMoveCnt1 = srv.writerMoveCount();
+
+ int prevNodeIdx = -1;
+
+ for (int iter = 0; iter < 10; iter++) {
+ int nodeIdx = rnd.nextInt(SRVS);
+
+ while (prevNodeIdx == nodeIdx)
+ nodeIdx = rnd.nextInt(SRVS);
+
+ prevNodeIdx = nodeIdx;
+
+ log.info("Iteration [iter=" + iter + ", node=" + nodeIdx + ']');
+
+ final long readMoveCnt = readMoveCnt1;
+ final long writeMoveCnt = writeMoveCnt1;
+
+ final int nodeIdx0 = nodeIdx;
+
+ GridTestUtils.waitForCondition(new GridAbsPredicate() {
+ @Override public boolean apply() {
+ byte[] data = new byte[100_000];
+
+ for (int j = 0; j < 10; j++) {
+ for (int i = 0; i < SRVS; i++) {
+ ClusterNode node = client.cluster().node(ignite(i).cluster().localNode().id());
+
+ IgniteCompute compute = client.compute(client.cluster().forNode(node));
+
+ compute.call(new DummyCallable(i == nodeIdx0 ? data : null));
+ }
+ }
+
+ return srv.readerMoveCount() > readMoveCnt && srv.writerMoveCount() > writeMoveCnt;
+ }
+ }, 30_000);
+
+ waitNioBalanceStop(Collections.singletonList(client), 30_000);
+
+ long readMoveCnt2 = srv.readerMoveCount();
+ long writeMoveCnt2 = srv.writerMoveCount();
+
+ log.info("Move counts [rc1=" + readMoveCnt1 +
+ ", wc1=" + writeMoveCnt1 +
+ ", rc2=" + readMoveCnt2 +
+ ", wc2=" + writeMoveCnt2 + ']');
+
+ assertTrue(readMoveCnt2 > readMoveCnt1);
+ assertTrue(writeMoveCnt2 > writeMoveCnt1);
+
+ readMoveCnt1 = readMoveCnt2;
+ writeMoveCnt1 = writeMoveCnt2;
+ }
+
+ waitNioBalanceStop(G.allGrids(), 10_000);
+ }
+ finally {
+ System.setProperty(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, "");
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testBalance2() throws Exception {
+ System.setProperty(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, "1000");
+
+ try {
+ startGridsMultiThreaded(5);
+
+ client = true;
+
+ startGridsMultiThreaded(5, 5);
+
+ for (int i = 0; i < 5; i++) {
+ log.info("Iteration: " + i);
+
+ final AtomicInteger idx = new AtomicInteger();
+
+ GridTestUtils.runMultiThreaded(new Callable<Void>() {
+ @Override public Void call() throws Exception {
+ Ignite node = ignite(idx.incrementAndGet() % 10);
+
+ ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+ int msgs = rnd.nextInt(500, 600);
+
+ for (int i = 0; i < msgs; i++) {
+ int sndTo = rnd.nextInt(10);
+
+ ClusterNode sntToNode = node.cluster().node(ignite(sndTo).cluster().localNode().id());
+
+ IgniteCompute compute = node.compute(node.cluster().forNode(sntToNode));
+
+ compute.call(new DummyCallable(new byte[rnd.nextInt(rnd.nextInt(256, 1024))]));
+ }
+
+ return null;
+ }
+ }, 30, "test-thread");
+
+ waitNioBalanceStop(G.allGrids(), 10_000);
+ }
+ }
+ finally {
+ System.setProperty(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, "");
+ }
+ }
+
+ /**
+ * @param nodes Node.
+ * @param timeout Timeout.
+ * @throws Exception If failed.
+ */
+ private void waitNioBalanceStop(List<Ignite> nodes, long timeout) throws Exception {
+ final List<GridNioServer> srvs = new ArrayList<>();
+
+ for (Ignite node : nodes) {
+ TcpCommunicationSpi spi = (TcpCommunicationSpi) node.configuration().getCommunicationSpi();
+
+ GridNioServer srv = GridTestUtils.getFieldValue(spi, "nioSrvr");
+
+ srvs.add(srv);
+ }
+
+ assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicateX() {
+ @Override public boolean applyx() throws IgniteCheckedException {
+ List<Long> rCnts = new ArrayList<>();
+ List<Long> wCnts = new ArrayList<>();
+
+ for (GridNioServer srv : srvs) {
+ long readerMovCnt1 = srv.readerMoveCount();
+ long writerMovCnt1 = srv.writerMoveCount();
+
+ rCnts.add(readerMovCnt1);
+ wCnts.add(writerMovCnt1);
+ }
+
+ U.sleep(2000);
+
+ for (int i = 0; i < srvs.size(); i++) {
+ GridNioServer srv = srvs.get(i);
+
+ long readerMovCnt1 = rCnts.get(i);
+ long writerMovCnt1 = wCnts.get(i);
+
+ long readerMovCnt2 = srv.readerMoveCount();
+ long writerMovCnt2 = srv.writerMoveCount();
+
+ if (readerMovCnt1 != readerMovCnt2) {
+ log.info("Readers balance is in progress [node=" + i + ", cnt1=" + readerMovCnt1 +
+ ", cnt2=" + readerMovCnt2 + ']');
+
+ return false;
+ }
+ if (writerMovCnt1 != writerMovCnt2) {
+ log.info("Writers balance is in progress [node=" + i + ", cnt1=" + writerMovCnt1 +
+ ", cnt2=" + writerMovCnt2 + ']');
+
+ return false;
+ }
+ }
+
+ return true;
+ }
+ }, timeout));
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testRandomBalance() throws Exception {
+ System.setProperty(GridNioServer.IGNITE_IO_BALANCE_RANDOM_BALANCE, "true");
+ System.setProperty(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, "500");
+
+ try {
+ final int NODES = 10;
+
+ startGridsMultiThreaded(NODES);
+
+ final long stopTime = System.currentTimeMillis() + 60_000;
+
+ GridTestUtils.runMultiThreaded(new Callable<Object>() {
+ @Override public Object call() throws Exception {
+ ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+ while (System.currentTimeMillis() < stopTime)
+ ignite(rnd.nextInt(NODES)).compute().broadcast(new DummyCallable(null));
+
+ return null;
+ }
+ }, 20, "test-thread");
+ }
+ finally {
+ System.setProperty(GridNioServer.IGNITE_IO_BALANCE_RANDOM_BALANCE, "");
+ System.setProperty(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, "");
+ }
+ }
+
+ /**
+ *
+ */
+ private static class DummyCallable implements IgniteCallable<Object> {
+ /** */
+ private byte[] data;
+
+ /**
+ * @param data Data.
+ */
+ DummyCallable(byte[] data) {
+ this.data = data;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object call() throws Exception {
+ return data;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessagesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessagesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessagesTest.java
new file mode 100644
index 0000000..b644878
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessagesTest.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.managers.communication;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class IgniteIoTestMessagesTest extends GridCommonAbstractTest {
+ /** */
+ private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+ /** */
+ private boolean client;
+
+ /** {@inheritDoc} */
+ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+ IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+ ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+ cfg.setClientMode(client);
+
+ return cfg;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void beforeTestsStarted() throws Exception {
+ super.beforeTestsStarted();
+
+ startGrids(3);
+
+ client = true;
+
+ startGrid(3);
+
+ startGrid(4);
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void afterTestsStopped() throws Exception {
+ stopAllGrids();
+
+ super.afterTestsStopped();
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testIoTestMessages() throws Exception {
+ for (Ignite node : G.allGrids()) {
+ IgniteKernal ignite = (IgniteKernal)node;
+
+ List<ClusterNode> rmts = new ArrayList<>(ignite.cluster().forRemotes().nodes());
+
+ assertEquals(4, rmts.size());
+
+ for (ClusterNode rmt : rmts) {
+ ignite.sendIoTest(rmt, new byte[1024], false);
+
+ ignite.sendIoTest(rmt, new byte[1024], true);
+
+ ignite.sendIoTest(rmts, new byte[1024], false);
+
+ ignite.sendIoTest(rmts, new byte[1024], true);
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteVariousConnectionNumberTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteVariousConnectionNumberTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteVariousConnectionNumberTest.java
new file mode 100644
index 0000000..510751e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteVariousConnectionNumberTest.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.managers.communication;
+
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ *
+ */
+public class IgniteVariousConnectionNumberTest extends GridCommonAbstractTest {
+ /** */
+ private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+ /** */
+ private static final int NODES = 6;
+
+ /** */
+ private static Random rnd = new Random();
+
+ /** */
+ private boolean client;
+
+ /** {@inheritDoc} */
+ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+ IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+ ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+ int connections = rnd.nextInt(10) + 1;
+
+ log.info("Node connections [name=" + gridName + ", connections=" + connections + ']');
+
+ ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setConnectionsPerNode(connections);
+ ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setUsePairedConnections(rnd.nextBoolean());
+ ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
+ cfg.setClientMode(client);
+
+ return cfg;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void beforeTestsStarted() throws Exception {
+ super.beforeTestsStarted();
+
+ long seed = U.currentTimeMillis();
+
+ rnd.setSeed(seed);
+
+ log.info("Random seed: " + seed);
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void afterTestsStopped() throws Exception {
+ stopAllGrids();
+
+ super.afterTestsStopped();
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testVariousConnectionNumber() throws Exception {
+ startGridsMultiThreaded(3);
+
+ client = true;
+
+ startGridsMultiThreaded(3, 3);
+
+ CacheConfiguration ccfg = new CacheConfiguration();
+
+ ccfg.setCacheMode(REPLICATED);
+ ccfg.setWriteSynchronizationMode(FULL_SYNC);
+
+ ignite(0).createCache(ccfg);
+
+ for (int i = 0; i < 10; i++) {
+ log.info("Iteration: " + i);
+
+ runOperations(5000);
+
+ awaitPartitionMapExchange();
+
+ int idx = ThreadLocalRandom.current().nextInt(NODES);
+
+ Ignite node = ignite(idx);
+
+ client = node.configuration().isClientMode();
+
+ stopGrid(idx);
+
+ startGrid(idx);
+ }
+ }
+
+ /**
+ * @param time Execution time.
+ * @throws Exception If failed.
+ */
+ private void runOperations(final long time) throws Exception {
+ final AtomicInteger idx = new AtomicInteger();
+
+ GridTestUtils.runMultiThreaded(new Callable<Void>() {
+ @Override public Void call() throws Exception {
+ Ignite node = ignite(idx.getAndIncrement() % NODES);
+
+ IgniteCache cache = node.cache(null);
+
+ long stopTime = U.currentTimeMillis() + time;
+
+ ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+ while (U.currentTimeMillis() < stopTime) {
+ cache.put(rnd.nextInt(10_000), 0);
+
+ node.compute().broadcast(new DummyJob());
+ }
+
+ return null;
+ }
+ }, NODES * 10, "test-thread");
+ }
+
+ /**
+ *
+ */
+ private static class DummyJob implements IgniteRunnable {
+ /** {@inheritDoc} */
+ @Override public void run() {
+ // No-op.
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
index 67ec371..eaa9923 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
@@ -86,6 +86,11 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
}
/** {@inheritDoc} */
+ @Override protected long getTestTimeout() {
+ return 6 * 60 * 1000;
+ }
+
+ /** {@inheritDoc} */
@Override protected void beforeTestsStarted() throws Exception {
super.beforeTestsStarted();
@@ -170,9 +175,17 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
}
/**
+ * @param cacheMode Cache mode.
+ * @param writeSync Write synchronization mode.
+ * @param fairAff Fair affinity flag.
+ * @param ignite Node to use.
+ * @param name Cache name.
*/
- protected void createCache(CacheMode cacheMode, CacheWriteSynchronizationMode writeSync, boolean fairAff,
- Ignite ignite, String name) {
+ protected void createCache(CacheMode cacheMode,
+ CacheWriteSynchronizationMode writeSync,
+ boolean fairAff,
+ Ignite ignite,
+ String name) {
ignite.createCache(cacheConfiguration(name, cacheMode, writeSync, fairAff));
}
@@ -269,9 +282,18 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
boolean checkData = fullSync && !optimistic;
+ long stopTime = System.currentTimeMillis() + 10_000;
+
for (int i = 0; i < 10_000; i++) {
- if (i % 100 == 0)
+ if (i % 100 == 0) {
+ if (System.currentTimeMillis() > stopTime) {
+ log.info("Stop on timeout, iteration: " + i);
+
+ break;
+ }
+
log.info("Iteration: " + i);
+ }
boolean rollback = i % 10 == 0;
@@ -557,4 +579,4 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
return old;
}
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
index 9405a19..3a2bc81 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
@@ -200,7 +200,9 @@ public abstract class GridAbstractCacheInterceptorRebalanceTest extends GridComm
private void testRebalance(final Operation operation) throws Exception {
interceptor = new RebalanceUpdateInterceptor();
- for (int iter = 0; iter < TEST_ITERATIONS; iter++) {
+ long stopTime = System.currentTimeMillis() + 2 * 60_000;
+
+ for (int iter = 0; iter < TEST_ITERATIONS && System.currentTimeMillis() < stopTime; iter++) {
log.info("Iteration: " + iter);
failed = false;
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateSelfTest.java
index 9458a63..6e2e91f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateSelfTest.java
@@ -115,10 +115,10 @@ public class GridCacheOffHeapMultiThreadedUpdateSelfTest extends GridCacheOffHea
if (gridCount() > 1)
testPutTx(keyForNode(1), PESSIMISTIC);
}
-
+
/**
* TODO: IGNITE-592.
- *
+ *
* @throws Exception If failed.
*/
public void testPutTxOptimistic() throws Exception {
@@ -227,4 +227,4 @@ public class GridCacheOffHeapMultiThreadedUpdateSelfTest extends GridCacheOffHea
assertFalse(failed);
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecovery10ConnectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecovery10ConnectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecovery10ConnectionsTest.java
new file mode 100644
index 0000000..30fc9ef
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecovery10ConnectionsTest.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed;
+
+/**
+ *
+ */
+public class IgniteCacheAtomicMessageRecovery10ConnectionsTest extends IgniteCacheAtomicMessageRecoveryTest {
+ /** {@inheritDoc} */
+ @Override protected int connectionsPerNode() {
+ return 10;
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest.java
new file mode 100644
index 0000000..71772ef
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest.java
@@ -0,0 +1,47 @@
+/*
+ * 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.distributed;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+
+/**
+ *
+ */
+public class IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest extends IgniteCacheAtomicMessageRecoveryTest {
+ /** {@inheritDoc} */
+ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+ IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+ TcpCommunicationSpi commSpi = (TcpCommunicationSpi)cfg.getCommunicationSpi();
+
+ assertTrue(commSpi.isUsePairedConnections());
+
+ commSpi.setUsePairedConnections(false);
+
+ return cfg;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected CacheAtomicityMode atomicityMode() {
+ return ATOMIC;
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheConnectionRecovery10ConnectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheConnectionRecovery10ConnectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheConnectionRecovery10ConnectionsTest.java
new file mode 100644
index 0000000..919aea6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheConnectionRecovery10ConnectionsTest.java
@@ -0,0 +1,35 @@
+/*
+ * 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.distributed;
+
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+
+/**
+ *
+ */
+public class IgniteCacheConnectionRecovery10ConnectionsTest extends IgniteCacheConnectionRecoveryTest {
+ /** {@inheritDoc} */
+ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+ IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+ ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setConnectionsPerNode(10);
+
+ return cfg;
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheCreatePutTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheCreatePutTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheCreatePutTest.java
index 2f700f3..a91de67 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheCreatePutTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheCreatePutTest.java
@@ -107,7 +107,7 @@ public class IgniteCacheCreatePutTest extends GridCommonAbstractTest {
try {
int iter = 0;
- while (System.currentTimeMillis() < stopTime) {
+ while (System.currentTimeMillis() < stopTime && iter < 5) {
log.info("Iteration: " + iter++);
try {
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java
index 0460a8f..1bfd727 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java
@@ -58,6 +58,7 @@ public abstract class IgniteCacheMessageRecoveryAbstractTest extends GridCommonA
commSpi.setSocketWriteTimeout(1000);
commSpi.setSharedMemoryPort(-1);
+ commSpi.setConnectionsPerNode(connectionsPerNode());
cfg.setCommunicationSpi(commSpi);
@@ -76,6 +77,13 @@ public abstract class IgniteCacheMessageRecoveryAbstractTest extends GridCommonA
}
/**
+ * @return Value for {@link TcpCommunicationSpi#setConnectionsPerNode(int)}.
+ */
+ protected int connectionsPerNode() {
+ return TcpCommunicationSpi.DFLT_CONN_PER_NODE;
+ }
+
+ /**
* @return Cache atomicity mode.
*/
protected abstract CacheAtomicityMode atomicityMode();
@@ -174,18 +182,22 @@ public abstract class IgniteCacheMessageRecoveryAbstractTest extends GridCommonA
static boolean closeSessions(Ignite ignite) throws Exception {
TcpCommunicationSpi commSpi = (TcpCommunicationSpi)ignite.configuration().getCommunicationSpi();
- Map<UUID, GridCommunicationClient> clients = U.field(commSpi, "clients");
+ Map<UUID, GridCommunicationClient[]> clients = U.field(commSpi, "clients");
boolean closed = false;
- for (GridCommunicationClient client : clients.values()) {
- GridTcpNioCommunicationClient client0 = (GridTcpNioCommunicationClient)client;
+ for (GridCommunicationClient[] clients0 : clients.values()) {
+ for (GridCommunicationClient client : clients0) {
+ if (client != null) {
+ GridTcpNioCommunicationClient client0 = (GridTcpNioCommunicationClient)client;
- GridNioSession ses = client0.session();
+ GridNioSession ses = client0.session();
- ses.close();
+ ses.close();
- closed = true;
+ closed = true;
+ }
+ }
}
return closed;
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageWriteTimeoutTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageWriteTimeoutTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageWriteTimeoutTest.java
index 6256225..0dd4079 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageWriteTimeoutTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageWriteTimeoutTest.java
@@ -50,8 +50,8 @@ public class IgniteCacheMessageWriteTimeoutTest extends GridCommonAbstractTest {
// Try provoke connection close on socket writeTimeout.
commSpi.setSharedMemoryPort(-1);
commSpi.setMessageQueueLimit(10);
- commSpi.setSocketReceiveBuffer(32);
- commSpi.setSocketSendBuffer(32);
+ commSpi.setSocketReceiveBuffer(40);
+ commSpi.setSocketSendBuffer(40);
commSpi.setSocketWriteTimeout(100);
commSpi.setUnacknowledgedMessagesBufferSize(1000);
commSpi.setConnectTimeout(10_000);
@@ -66,15 +66,20 @@ public class IgniteCacheMessageWriteTimeoutTest extends GridCommonAbstractTest {
super.afterTest();
}
+ /** {@inheritDoc} */
+ @Override protected long getTestTimeout() {
+ return 10 * 60_000;
+ }
+
/**
* @throws Exception If failed.
*/
public void testMessageQueueLimit() throws Exception {
- startGridsMultiThreaded(3);
-
- for (int i = 0; i < 15; i++) {
+ for (int i = 0; i < 3; i++) {
log.info("Iteration: " + i);
+ startGridsMultiThreaded(3);
+
IgniteInternalFuture<?> fut1 = startJobThreads(50);
U.sleep(100);
@@ -83,6 +88,8 @@ public class IgniteCacheMessageWriteTimeoutTest extends GridCommonAbstractTest {
fut1.get();
fut2.get();
+
+ stopAllGrids();
}
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
index 3fca826..322690c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
@@ -86,7 +86,6 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
plc.setMaxSize(100000);
ccfg.setEvictionPolicy(plc);
- ccfg.setEvictSynchronized(true);
c.setCacheConfiguration(ccfg);
@@ -95,6 +94,11 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
return c;
}
+ /** {@inheritDoc} */
+ @Override protected long getTestTimeout() {
+ return 60_000;
+ }
+
/**
* @throws Exception If failed.
*/
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest.java
new file mode 100644
index 0000000..e8175e5
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest.java
@@ -0,0 +1,35 @@
+/*
+ * 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.distributed.near;
+
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ *
+ */
+public class GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest extends
+ GridCacheAtomicPrimaryWriteOrderMultiNodeFullApiSelfTest {
+ /** {@inheritDoc} */
+ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+ IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+ cfg.setStripedPoolSize(-1);
+
+ return cfg;
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest.java
new file mode 100644
index 0000000..05fe85f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest.java
@@ -0,0 +1,35 @@
+/*
+ * 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.distributed.near;
+
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ *
+ */
+public class GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest extends
+ GridCachePartitionedMultiNodeFullApiSelfTest {
+ /** {@inheritDoc} */
+ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+ IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+ cfg.setStripedPoolSize(-1);
+
+ return cfg;
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java
index c9d18eb..e9d74ff 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java
@@ -211,7 +211,7 @@ public class TxDeadlockDetectionNoHangsTest extends GridCommonAbstractTest {
tx.commit();
}
catch (Exception e) {
- e.printStackTrace();
+ log.info("Ignore error: " + e);
}
}
}, NODES_CNT * 3, "tx-thread");
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
index aa240aa..f6a06c2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
@@ -111,6 +111,9 @@ public class TxOptimisticDeadlockDetectionTest extends GridCommonAbstractTest {
cfg.setClientMode(client);
+ // Test spi blocks message send, this can cause hang with striped pool.
+ cfg.setStripedPoolSize(-1);
+
return cfg;
}
@@ -274,8 +277,8 @@ public class TxOptimisticDeadlockDetectionTest extends GridCommonAbstractTest {
Object k;
- log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() +
- ", tx=" + tx + ", key=" + transformer.apply(key) + ']');
+ log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode().id() +
+ ", tx=" + tx.xid() + ", key=" + transformer.apply(key) + ']');
cache.put(transformer.apply(key), 0);
@@ -309,23 +312,27 @@ public class TxOptimisticDeadlockDetectionTest extends GridCommonAbstractTest {
entries.put(k, 2);
}
- log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() +
- ", tx=" + tx + ", entries=" + entries + ']');
+ log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode().id() +
+ ", tx=" + tx.xid() + ", entries=" + entries + ']');
cache.putAll(entries);
tx.commit();
}
catch (Throwable e) {
- U.error(log, "Expected exception: ", e);
+ log.info("Expected exception: " + e);
+
+ e.printStackTrace(System.out);
// At least one stack trace should contain TransactionDeadlockException.
if (hasCause(e, TransactionTimeoutException.class) &&
- hasCause(e, TransactionDeadlockException.class)
- ) {
- if (deadlockErr.compareAndSet(null, cause(e, TransactionDeadlockException.class)))
- U.error(log, "At least one stack trace should contain " +
- TransactionDeadlockException.class.getSimpleName(), e);
+ hasCause(e, TransactionDeadlockException.class)) {
+ if (deadlockErr.compareAndSet(null, cause(e, TransactionDeadlockException.class))) {
+ log.info("At least one stack trace should contain " +
+ TransactionDeadlockException.class.getSimpleName());
+
+ e.printStackTrace(System.out);
+ }
}
}
}
@@ -344,7 +351,7 @@ public class TxOptimisticDeadlockDetectionTest extends GridCommonAbstractTest {
TransactionDeadlockException deadlockE = deadlockErr.get();
- assertNotNull(deadlockE);
+ assertNotNull("Failed to detect deadlock", deadlockE);
boolean fail = false;
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java
index 6fc7e02..7b5abf5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java
@@ -372,4 +372,4 @@ public class GridServiceProcessorProxySelfTest extends GridServiceProcessorAbstr
X.println("Executing cache service: " + ctx.name());
}
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java
index adcd144..4bc9f01 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.util.future;
import java.util.concurrent.Callable;
import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
@@ -227,87 +228,98 @@ public class GridFutureAdapterSelfTest extends GridCommonAbstractTest {
*
* @throws Exception In case of any exception.
*/
- @SuppressWarnings("ErrorNotRethrown")
public void testChaining() throws Exception {
+ checkChaining(null);
+
+ ExecutorService exec = Executors.newFixedThreadPool(1);
+
+ try {
+ checkChaining(exec);
+
+ GridFinishedFuture<Integer> fut = new GridFinishedFuture<>(1);
+
+ IgniteInternalFuture<Object> chain = fut.chain(new CX1<IgniteInternalFuture<Integer>, Object>() {
+ @Override public Object applyx(IgniteInternalFuture<Integer> fut) throws IgniteCheckedException {
+ return fut.get() + 1;
+ }
+ }, exec);
+
+ assertEquals(2, chain.get());
+ }
+ finally {
+ exec.shutdown();
+ }
+ }
+
+ /**
+ * @param exec Executor for chain callback.
+ * @throws Exception If failed.
+ */
+ @SuppressWarnings("ErrorNotRethrown")
+ private void checkChaining(ExecutorService exec) throws Exception {
final CX1<IgniteInternalFuture<Object>, Object> passThrough = new CX1<IgniteInternalFuture<Object>, Object>() {
@Override public Object applyx(IgniteInternalFuture<Object> f) throws IgniteCheckedException {
return f.get();
}
};
- final GridTestKernalContext ctx = new GridTestKernalContext(log);
-
- ctx.setExecutorService(Executors.newFixedThreadPool(1));
- ctx.setSystemExecutorService(Executors.newFixedThreadPool(1));
-
- ctx.add(new PoolProcessor(ctx));
- ctx.add(new GridClosureProcessor(ctx));
+ GridFutureAdapter<Object> fut = new GridFutureAdapter<>();
+ IgniteInternalFuture<Object> chain = exec != null ? fut.chain(passThrough, exec) : fut.chain(passThrough);
- ctx.start();
+ assertFalse(fut.isDone());
+ assertFalse(chain.isDone());
try {
- // Test result returned.
-
- GridFutureAdapter<Object> fut = new GridFutureAdapter<>();
- IgniteInternalFuture<Object> chain = fut.chain(passThrough);
+ chain.get(20);
- assertFalse(fut.isDone());
- assertFalse(chain.isDone());
-
- try {
- chain.get(20);
-
- fail("Expects timeout exception.");
- }
- catch (IgniteFutureTimeoutCheckedException e) {
- info("Expected timeout exception: " + e.getMessage());
- }
+ fail("Expects timeout exception.");
+ }
+ catch (IgniteFutureTimeoutCheckedException e) {
+ info("Expected timeout exception: " + e.getMessage());
+ }
- fut.onDone("result");
+ fut.onDone("result");
- assertEquals("result", chain.get(1));
+ assertEquals("result", chain.get(1));
- // Test exception re-thrown.
+ // Test exception re-thrown.
- fut = new GridFutureAdapter<>();
- chain = fut.chain(passThrough);
+ fut = new GridFutureAdapter<>();
+ chain = exec != null ? fut.chain(passThrough, exec) : fut.chain(passThrough);
- fut.onDone(new ClusterGroupEmptyCheckedException("test exception"));
+ fut.onDone(new ClusterGroupEmptyCheckedException("test exception"));
- try {
- chain.get();
+ try {
+ chain.get();
- fail("Expects failed with exception.");
- }
- catch (ClusterGroupEmptyCheckedException e) {
- info("Expected exception: " + e.getMessage());
- }
+ fail("Expects failed with exception.");
+ }
+ catch (ClusterGroupEmptyCheckedException e) {
+ info("Expected exception: " + e.getMessage());
+ }
- // Test error re-thrown.
+ // Test error re-thrown.
- fut = new GridFutureAdapter<>();
- chain = fut.chain(passThrough);
+ fut = new GridFutureAdapter<>();
+ chain = exec != null ? fut.chain(passThrough, exec) : fut.chain(passThrough);
- try {
- fut.onDone(new StackOverflowError("test error"));
+ try {
+ fut.onDone(new StackOverflowError("test error"));
+ if (exec == null)
fail("Expects failed with error.");
- }
- catch (StackOverflowError e) {
- info("Expected error: " + e.getMessage());
- }
+ }
+ catch (StackOverflowError e) {
+ info("Expected error: " + e.getMessage());
+ }
- try {
- chain.get();
+ try {
+ chain.get();
- fail("Expects failed with error.");
- }
- catch (StackOverflowError e) {
- info("Expected error: " + e.getMessage());
- }
+ fail("Expects failed with error.");
}
- finally {
- ctx.stop(false);
+ catch (StackOverflowError e) {
+ info("Expected error: " + e.getMessage());
}
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/util/nio/impl/GridNioFilterChainSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/impl/GridNioFilterChainSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/impl/GridNioFilterChainSelfTest.java
index 201fd27..d403784 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/impl/GridNioFilterChainSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/impl/GridNioFilterChainSelfTest.java
@@ -114,7 +114,7 @@ public class GridNioFilterChainSelfTest extends GridCommonAbstractTest {
proceedExceptionCaught(ses, ex);
}
- @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) {
+ @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) {
sndEvt.compareAndSet(null, ses.<String>meta(MESSAGE_WRITE_META_NAME));
sndMsgObj.compareAndSet(null, msg);
@@ -155,7 +155,7 @@ public class GridNioFilterChainSelfTest extends GridCommonAbstractTest {
chain.onSessionIdleTimeout(ses);
chain.onSessionWriteTimeout(ses);
assertNull(chain.onSessionClose(ses));
- assertNull(chain.onSessionWrite(ses, snd));
+ assertNull(chain.onSessionWrite(ses, snd, true));
assertEquals("DCBA", connectedEvt.get());
assertEquals("DCBA", disconnectedEvt.get());
@@ -210,10 +210,10 @@ public class GridNioFilterChainSelfTest extends GridCommonAbstractTest {
}
/** {@inheritDoc} */
- @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
+ @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) throws IgniteCheckedException {
chainMeta(ses, MESSAGE_WRITE_META_NAME);
- return proceedSessionWrite(ses, msg);
+ return proceedSessionWrite(ses, msg, fut);
}
/** {@inheritDoc} */
@@ -349,6 +349,11 @@ public class GridNioFilterChainSelfTest extends GridCommonAbstractTest {
}
/** {@inheritDoc} */
+ @Override public void sendNoFuture(Object msg) {
+ // No-op.
+ }
+
+ /** {@inheritDoc} */
@Override public GridNioFuture<Object> resumeReads() {
return null;
}
@@ -369,13 +374,28 @@ public class GridNioFilterChainSelfTest extends GridCommonAbstractTest {
}
/** {@inheritDoc} */
- @Override public void recoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+ @Override public void outRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+ // No-op.
+ }
+
+ /** {@inheritDoc} */
+ @Nullable @Override public GridNioRecoveryDescriptor outRecoveryDescriptor() {
+ return null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void inRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
// No-op.
}
/** {@inheritDoc} */
- @Nullable @Override public GridNioRecoveryDescriptor recoveryDescriptor() {
+ @Nullable @Override public GridNioRecoveryDescriptor inRecoveryDescriptor() {
return null;
}
+
+ /** {@inheritDoc} */
+ @Override public void systemMessage(Object msg) {
+ // No-op.
+ }
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/loadtests/nio/GridNioBenchmarkClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/nio/GridNioBenchmarkClient.java b/modules/core/src/test/java/org/apache/ignite/loadtests/nio/GridNioBenchmarkClient.java
index 61a13b1..25dd780 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/nio/GridNioBenchmarkClient.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/nio/GridNioBenchmarkClient.java
@@ -83,7 +83,7 @@ public class GridNioBenchmarkClient {
*/
public void run() throws IOException, InterruptedException {
for (int i = 0; i < connCnt; i++)
- exec.submit(new ClientThread());
+ exec.execute(new ClientThread());
Thread.sleep(5*60*1000);
@@ -167,4 +167,4 @@ public class GridNioBenchmarkClient {
return read;
}
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PRecursionTaskSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PRecursionTaskSelfTest.java b/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PRecursionTaskSelfTest.java
index f21f31b..a18ef32 100644
--- a/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PRecursionTaskSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PRecursionTaskSelfTest.java
@@ -196,4 +196,4 @@ public class GridP2PRecursionTaskSelfTest extends GridCommonAbstractTest {
return ignite.compute().execute(FactorialTask.class, arg);
}
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java
index 652e47f..5ca8f26 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java
@@ -69,7 +69,7 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
private static final int commExtPort2 = 20100;
/** */
- private AddressResolver resolver;
+ private AddressResolver rslvr;
/** */
private boolean ipFinderUseLocPorts;
@@ -111,14 +111,15 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
cfg.setConnectorConfiguration(null);
TcpCommunicationSpi commSpi = new TcpCommunicationSpi() {
- @Override protected GridCommunicationClient createTcpClient(ClusterNode node) throws IgniteCheckedException {
+ @Override protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx)
+ throws IgniteCheckedException {
Map<String, Object> attrs = new HashMap<>(node.attributes());
attrs.remove(createSpiAttributeName(ATTR_PORT));
((TcpDiscoveryNode)node).setAttributes(attrs);
- return super.createTcpClient(node);
+ return super.createTcpClient(node, connIdx);
}
};
@@ -126,12 +127,13 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
commSpi.setLocalPort(commLocPort);
commSpi.setLocalPortRange(1);
commSpi.setSharedMemoryPort(-1);
+ commSpi.setConnectionsPerNode(1);
cfg.setCommunicationSpi(commSpi);
- assert resolver != null;
+ assert rslvr != null;
- cfg.setAddressResolver(resolver);
+ cfg.setAddressResolver(rslvr);
return cfg;
}
@@ -147,7 +149,7 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
map.put(new InetSocketAddress("127.0.0.1", locPort2), F.asList(new InetSocketAddress("127.0.0.1", extPort2)));
map.put(new InetSocketAddress("127.0.0.1", commLocPort2), F.asList(new InetSocketAddress("127.0.0.1", commExtPort2)));
- resolver = new AddressResolver() {
+ rslvr = new AddressResolver() {
@Override public Collection<InetSocketAddress> getExternalAddresses(InetSocketAddress addr) {
return map.get(addr);
}
@@ -167,7 +169,7 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
map.put("127.0.0.1:" + locPort2, "127.0.0.1:" + extPort2);
map.put("127.0.0.1:" + commLocPort2, "127.0.0.1:" + commExtPort2);
- resolver = new BasicAddressResolver(map);
+ rslvr = new BasicAddressResolver(map);
doTestForward();
}
@@ -180,7 +182,7 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
map.put("127.0.0.1", "127.0.0.1");
- resolver = new BasicAddressResolver(map);
+ rslvr = new BasicAddressResolver(map);
ipFinderUseLocPorts = true;
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
index 076724d..3c4fea0 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
@@ -90,16 +90,36 @@ abstract class GridTcpCommunicationSpiAbstractTest extends GridAbstractCommunica
super.afterTest();
for (CommunicationSpi spi : spis.values()) {
- ConcurrentMap<UUID, GridCommunicationClient> clients = U.field(spi, "clients");
+ ConcurrentMap<UUID, GridCommunicationClient[]> clients = U.field(spi, "clients");
+
+ for (int i = 0; i < 20; i++) {
+ GridCommunicationClient client0 = null;
+
+ for (GridCommunicationClient[] clients0 : clients.values()) {
+ for (GridCommunicationClient client : clients0) {
+ if (client != null) {
+ client0 = client;
+
+ break;
+ }
+ }
+
+ if (client0 != null)
+ break;
+ }
+
+ if (client0 == null)
+ return;
- for (int i = 0; i < 20 && !clients.isEmpty(); i++) {
info("Check failed for SPI [grid=" +
- GridTestUtils.getFieldValue(spi, IgniteSpiAdapter.class, "gridName") + ", spi=" + spi + ']');
+ GridTestUtils.getFieldValue(spi, IgniteSpiAdapter.class, "gridName") +
+ ", client=" + client0 +
+ ", spi=" + spi + ']');
U.sleep(1000);
}
- assert clients.isEmpty() : "Clients: " + clients;
+ fail("Failed to wait when clients are closed.");
}
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
index 8635d94..a649130 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
@@ -31,6 +31,7 @@ import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.cluster.ClusterNode;
import org.apache.ignite.configuration.IgniteConfiguration;
@@ -83,6 +84,12 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
/** Use ssl. */
protected boolean useSsl;
+ /** */
+ private int connectionsPerNode = 1;
+
+ /** */
+ private boolean pairedConnections = true;
+
/**
*
*/
@@ -163,6 +170,34 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
/**
* @throws Exception If failed.
*/
+ public void testMultithreaded_10Connections() throws Exception {
+ connectionsPerNode = 10;
+
+ testMultithreaded();
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testMultithreaded_NoPairedConnections() throws Exception {
+ pairedConnections = false;
+
+ testMultithreaded();
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testMultithreaded_10ConnectionsNoPaired() throws Exception {
+ pairedConnections = false;
+ connectionsPerNode = 10;
+
+ testMultithreaded();
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
public void testWithLoad() throws Exception {
int threads = Runtime.getRuntime().availableProcessors() * 5;
@@ -244,7 +279,7 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
final AtomicInteger idx = new AtomicInteger();
try {
- GridTestUtils.runMultiThreaded(new Callable<Void>() {
+ final Callable<Void> c = new Callable<Void>() {
@Override public Void call() throws Exception {
int idx0 = idx.getAndIncrement();
@@ -270,7 +305,40 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
return null;
}
- }, threads, "test");
+ };
+
+ List<Thread> threadsList = new ArrayList<>();
+
+ final AtomicBoolean fail = new AtomicBoolean();
+
+ final AtomicLong tId = new AtomicLong();
+
+ for (int t = 0; t < threads; t++) {
+ Thread t0 = new Thread(new Runnable() {
+ @Override public void run() {
+ try {
+ c.call();
+ }
+ catch (Throwable e) {
+ log.error("Unexpected error: " + e, e);
+
+ fail.set(true);
+ }
+ }
+ }) {
+ @Override public long getId() {
+ // Override getId to use all connections.
+ return tId.getAndIncrement();
+ }
+ };
+
+ threadsList.add(t0);
+
+ t0.start();
+ }
+
+ for (Thread t0 : threadsList)
+ t0.join();
assertTrue(latch.await(10, TimeUnit.SECONDS));
@@ -281,17 +349,19 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
final GridNioServer srv = U.field(spi, "nioSrvr");
+ final int conns = pairedConnections ? 2 : 1;
+
GridTestUtils.waitForCondition(new GridAbsPredicate() {
@Override public boolean apply() {
Collection sessions = U.field(srv, "sessions");
- return sessions.size() == 1;
+ return sessions.size() == conns * connectionsPerNode;
}
}, 5000);
Collection sessions = U.field(srv, "sessions");
- assertEquals(1, sessions.size());
+ assertEquals(conns * connectionsPerNode, sessions.size());
}
assertEquals(expMsgs, lsnr.cntr.get());
@@ -320,6 +390,8 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
spi.setIdleConnectionTimeout(60_000);
spi.setConnectTimeout(10_000);
spi.setSharedMemoryPort(-1);
+ spi.setConnectionsPerNode(connectionsPerNode);
+ spi.setUsePairedConnections(pairedConnections);
return spi;
}
@@ -434,4 +506,4 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
rsrcs.stopThreads();
}
-}
\ No newline at end of file
+}
[19/25] ignite git commit: IGNITE-3220 I/O bottleneck on
server/client cluster configuration Communications optimizations: -
possibility to open separate in/out connections - possibility to have
multiple connections between nodes - implemented NIO sessio
Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index c8e2e0b..bc1f173 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -37,6 +37,7 @@ import java.nio.channels.WritableByteChannel;
import java.nio.channels.spi.SelectorProvider;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collection;
import java.util.Deque;
import java.util.Iterator;
import java.util.List;
@@ -44,6 +45,8 @@ import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicLong;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteException;
import org.apache.ignite.IgniteLogger;
@@ -63,6 +66,7 @@ import org.apache.ignite.internal.util.worker.GridWorker;
import org.apache.ignite.lang.IgniteBiInClosure;
import org.apache.ignite.lang.IgniteInClosure;
import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.lang.IgniteRunnable;
import org.apache.ignite.plugin.extensions.communication.Message;
import org.apache.ignite.plugin.extensions.communication.MessageReader;
import org.apache.ignite.plugin.extensions.communication.MessageWriter;
@@ -86,11 +90,14 @@ import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.NIO_OPER
*
*/
public class GridNioServer<T> {
+ /** */
+ public static final String IGNITE_IO_BALANCE_RANDOM_BALANCE = "IGNITE_IO_BALANCE_RANDOM_BALANCER";
+
/** Default session write timeout. */
public static final int DFLT_SES_WRITE_TIMEOUT = 5000;
/** Default send queue limit. */
- public static final int DFLT_SEND_QUEUE_LIMIT = 1024;
+ public static final int DFLT_SEND_QUEUE_LIMIT = 0;
/** Time, which server will wait before retry operation. */
private static final long ERR_WAIT_TIME = 2000;
@@ -122,6 +129,9 @@ public class GridNioServer<T> {
}
}
+ /** Defines how many times selector should do {@code selectNow()} before doing {@code select(long)}. */
+ private long selectorSpins;
+
/** Accept worker thread. */
@GridToStringExclude
private final IgniteThread acceptThread;
@@ -145,9 +155,13 @@ public class GridNioServer<T> {
/** Flag indicating if this server should use direct buffers. */
private final boolean directBuf;
- /** Index to select which thread will serve next socket channel. Using round-robin balancing. */
+ /** Index to select which thread will serve next incoming socket channel. Using round-robin balancing. */
+ @GridToStringExclude
+ private int readBalanceIdx;
+
+ /** Index to select which thread will serve next out socket channel. Using round-robin balancing. */
@GridToStringExclude
- private int balanceIdx;
+ private int writeBalanceIdx = 1;
/** Tcp no delay flag. */
private final boolean tcpNoDelay;
@@ -204,12 +218,25 @@ public class GridNioServer<T> {
/** Optional listener to monitor outbound message queue size. */
private IgniteBiInClosure<GridNioSession, Integer> msgQueueLsnr;
+ /** */
+ private final AtomicLong readerMoveCnt = new AtomicLong();
+
+ /** */
+ private final AtomicLong writerMoveCnt = new AtomicLong();
+
+ /** */
+ private final IgniteRunnable balancer;
+
/**
* @param addr Address.
* @param port Port.
* @param log Log.
* @param selectorCnt Count of selectors and selecting threads.
* @param gridName Grid name.
+ * @param srvName Logical server name for threads identification.
+ * @param selectorSpins Defines how many non-blocking {@code selector.selectNow()} should be made before
+ * falling into {@code selector.select(long)} in NIO server. Long value. Default is {@code 0}.
+ * Can be set to {@code Long.MAX_VALUE} so selector threads will never block.
* @param tcpNoDelay If TCP_NODELAY option should be set to accepted sockets.
* @param directBuf Direct buffer flag.
* @param order Byte order.
@@ -223,6 +250,7 @@ public class GridNioServer<T> {
* @param writerFactory Writer factory.
* @param skipRecoveryPred Skip recovery predicate.
* @param msgQueueLsnr Message queue size listener.
+ * @param balancing NIO sessions balancing flag.
* @param filters Filters for this server.
* @throws IgniteCheckedException If failed.
*/
@@ -232,6 +260,8 @@ public class GridNioServer<T> {
IgniteLogger log,
int selectorCnt,
@Nullable String gridName,
+ @Nullable String srvName,
+ long selectorSpins,
boolean tcpNoDelay,
boolean directBuf,
ByteOrder order,
@@ -245,6 +275,7 @@ public class GridNioServer<T> {
GridNioMessageWriterFactory writerFactory,
IgnitePredicate<Message> skipRecoveryPred,
IgniteBiInClosure<GridNioSession, Integer> msgQueueLsnr,
+ boolean balancing,
GridNioFilter... filters
) throws IgniteCheckedException {
if (port != -1)
@@ -268,6 +299,7 @@ public class GridNioServer<T> {
this.sockSndBuf = sockSndBuf;
this.sndQueueLimit = sndQueueLimit;
this.msgQueueLsnr = msgQueueLsnr;
+ this.selectorSpins = selectorSpins;
filterChain = new GridNioFilterChain<>(log, lsnr, new HeadFilter(), filters);
@@ -299,9 +331,16 @@ public class GridNioServer<T> {
clientThreads = new IgniteThread[selectorCnt];
for (int i = 0; i < selectorCnt; i++) {
+ String threadName;
+
+ if (srvName == null)
+ threadName = "grid-nio-worker-" + i;
+ else
+ threadName = "grid-nio-worker-" + srvName + "-" + i;
+
AbstractNioClientWorker worker = directMode ?
- new DirectNioClientWorker(i, gridName, "grid-nio-worker-" + i, log) :
- new ByteBufferNioClientWorker(i, gridName, "grid-nio-worker-" + i, log);
+ new DirectNioClientWorker(i, gridName, threadName, log) :
+ new ByteBufferNioClientWorker(i, gridName, threadName, log);
clientWorkers.add(worker);
@@ -315,6 +354,32 @@ public class GridNioServer<T> {
this.writerFactory = writerFactory;
this.skipRecoveryPred = skipRecoveryPred != null ? skipRecoveryPred : F.<Message>alwaysFalse();
+
+ long balancePeriod = IgniteSystemProperties.getLong(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, 5000);
+
+ IgniteRunnable balancer0 = null;
+
+ if (balancing && balancePeriod > 0) {
+ boolean rndBalance = IgniteSystemProperties.getBoolean(IGNITE_IO_BALANCE_RANDOM_BALANCE, false);
+
+ balancer0 = rndBalance ? new RandomBalancer() : new SizeBasedBalancer(balancePeriod);
+ }
+
+ this.balancer = balancer0;
+ }
+
+ /**
+ * @return Number of reader sessions move.
+ */
+ public long readerMoveCount() {
+ return readerMoveCnt.get();
+ }
+
+ /**
+ * @return Number of reader writer move.
+ */
+ public long writerMoveCount() {
+ return writerMoveCnt.get();
}
/**
@@ -377,6 +442,13 @@ public class GridNioServer<T> {
}
/**
+ * @return Selector spins.
+ */
+ public long selectorSpins() {
+ return selectorSpins;
+ }
+
+ /**
* @param ses Session to close.
* @return Future for operation.
*/
@@ -390,7 +462,7 @@ public class GridNioServer<T> {
NioOperationFuture<Boolean> fut = new NioOperationFuture<>(impl, NioOperation.CLOSE);
- clientWorkers.get(impl.selectorIndex()).offer(fut);
+ impl.offerStateChange(fut);
return fut;
}
@@ -398,61 +470,91 @@ public class GridNioServer<T> {
/**
* @param ses Session.
* @param msg Message.
+ * @param createFut {@code True} if future should be created.
* @return Future for operation.
*/
- GridNioFuture<?> send(GridNioSession ses, ByteBuffer msg) {
- assert ses instanceof GridSelectorNioSessionImpl;
+ GridNioFuture<?> send(GridNioSession ses, ByteBuffer msg, boolean createFut) throws IgniteCheckedException {
+ assert ses instanceof GridSelectorNioSessionImpl : ses;
GridSelectorNioSessionImpl impl = (GridSelectorNioSessionImpl)ses;
- NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg);
+ if (createFut) {
+ NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg);
- send0(impl, fut, false);
+ send0(impl, fut, false);
- return fut;
+ return fut;
+ }
+ else {
+ SessionWriteRequest req = new WriteRequestImpl(ses, msg, true);
+
+ send0(impl, req, false);
+
+ return null;
+ }
}
/**
* @param ses Session.
* @param msg Message.
+ * @param createFut {@code True} if future should be created.
* @return Future for operation.
*/
- GridNioFuture<?> send(GridNioSession ses, Message msg) {
+ GridNioFuture<?> send(GridNioSession ses, Message msg, boolean createFut) throws IgniteCheckedException {
assert ses instanceof GridSelectorNioSessionImpl;
GridSelectorNioSessionImpl impl = (GridSelectorNioSessionImpl)ses;
- NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg,
- skipRecoveryPred.apply(msg));
+ if (createFut) {
+ NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg,
+ skipRecoveryPred.apply(msg));
- send0(impl, fut, false);
+ send0(impl, fut, false);
- return fut;
+ return fut;
+ }
+ else {
+ SessionWriteRequest req = new WriteRequestImpl(ses, msg, skipRecoveryPred.apply(msg));
+
+ send0(impl, req, false);
+
+ return null;
+ }
}
/**
* @param ses Session.
- * @param fut Future.
+ * @param req Request.
* @param sys System message flag.
+ * @throws IgniteCheckedException If session was closed.
*/
- private void send0(GridSelectorNioSessionImpl ses, NioOperationFuture<?> fut, boolean sys) {
+ private void send0(GridSelectorNioSessionImpl ses, SessionWriteRequest req, boolean sys) throws IgniteCheckedException {
assert ses != null;
- assert fut != null;
+ assert req != null;
- int msgCnt = sys ? ses.offerSystemFuture(fut) : ses.offerFuture(fut);
+ int msgCnt = sys ? ses.offerSystemFuture(req) : ses.offerFuture(req);
IgniteInClosure<IgniteException> ackC;
if (!sys && (ackC = ses.removeMeta(ACK_CLOSURE.ordinal())) != null)
- fut.ackClosure(ackC);
+ req.ackClosure(ackC);
if (ses.closed()) {
- if (ses.removeFuture(fut))
- fut.connectionClosed();
+ if (ses.removeFuture(req)) {
+ IOException err = new IOException("Failed to send message (connection was closed): " + ses);
+
+ req.onError(err);
+
+ if (!(req instanceof GridNioFuture))
+ throw new IgniteCheckedException(err);
+ }
+ }
+ else if (!ses.procWrite.get() && ses.procWrite.compareAndSet(false, true)) {
+ AbstractNioClientWorker worker = (AbstractNioClientWorker)ses.worker();
+
+ if (worker != null)
+ worker.offer((SessionChangeRequest)req);
}
- else if (msgCnt == 1)
- // Change from 0 to 1 means that worker thread should be waken up.
- clientWorkers.get(ses.selectorIndex()).offer(fut);
if (msgQueueLsnr != null)
msgQueueLsnr.apply(ses, msgCnt);
@@ -463,10 +565,10 @@ public class GridNioServer<T> {
*
* @param ses Session.
* @param msg Message.
- * @return Future.
+ * @throws IgniteCheckedException If session was closed.
*/
- public GridNioFuture<?> sendSystem(GridNioSession ses, Message msg) {
- return sendSystem(ses, msg, null);
+ public void sendSystem(GridNioSession ses, Message msg) throws IgniteCheckedException {
+ sendSystem(ses, msg, null);
}
/**
@@ -475,27 +577,30 @@ public class GridNioServer<T> {
* @param ses Session.
* @param msg Message.
* @param lsnr Future listener notified from the session thread.
- * @return Future.
+ * @throws IgniteCheckedException If session was closed.
*/
- public GridNioFuture<?> sendSystem(GridNioSession ses,
+ public void sendSystem(GridNioSession ses,
Message msg,
- @Nullable IgniteInClosure<? super IgniteInternalFuture<?>> lsnr) {
+ @Nullable IgniteInClosure<? super IgniteInternalFuture<?>> lsnr) throws IgniteCheckedException {
assert ses instanceof GridSelectorNioSessionImpl;
GridSelectorNioSessionImpl impl = (GridSelectorNioSessionImpl)ses;
- NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg,
- skipRecoveryPred.apply(msg));
-
if (lsnr != null) {
+ NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg,
+ skipRecoveryPred.apply(msg));
+
fut.listen(lsnr);
assert !fut.isDone();
- }
- send0(impl, fut, true);
+ send0(impl, fut, true);
+ }
+ else {
+ SessionWriteRequest req = new WriteRequestSystemImpl(ses, msg);
- return fut;
+ send0(impl, req, true);
+ }
}
/**
@@ -504,37 +609,69 @@ public class GridNioServer<T> {
public void resend(GridNioSession ses) {
assert ses instanceof GridSelectorNioSessionImpl;
- GridNioRecoveryDescriptor recoveryDesc = ses.recoveryDescriptor();
+ GridNioRecoveryDescriptor recoveryDesc = ses.outRecoveryDescriptor();
- if (recoveryDesc != null && !recoveryDesc.messagesFutures().isEmpty()) {
- Deque<GridNioFuture<?>> futs = recoveryDesc.messagesFutures();
+ if (recoveryDesc != null && !recoveryDesc.messagesRequests().isEmpty()) {
+ Deque<SessionWriteRequest> futs = recoveryDesc.messagesRequests();
if (log.isDebugEnabled())
log.debug("Resend messages [rmtNode=" + recoveryDesc.node().id() + ", msgCnt=" + futs.size() + ']');
GridSelectorNioSessionImpl ses0 = (GridSelectorNioSessionImpl)ses;
- GridNioFuture<?> fut0 = futs.iterator().next();
+ SessionWriteRequest fut0 = futs.iterator().next();
- for (GridNioFuture<?> fut : futs) {
+ for (SessionWriteRequest fut : futs) {
fut.messageThread(true);
- ((NioOperationFuture)fut).resetSession(ses0);
+ fut.resetSession(ses0);
}
ses0.resend(futs);
// Wake up worker.
- clientWorkers.get(ses0.selectorIndex()).offer(((NioOperationFuture)fut0));
+ ses0.offerStateChange((GridNioServer.SessionChangeRequest)fut0);
}
}
/**
+ * @return Sessions.
+ */
+ public Collection<? extends GridNioSession> sessions() {
+ return sessions;
+ }
+
+ /**
+ * @return Workers.
+ */
+ public List<AbstractNioClientWorker> workers() {
+ return clientWorkers;
+ }
+
+ /**
+ * @param ses Session.
+ * @param from Move from index.
+ * @param to Move to index.
+ */
+ private void moveSession(GridNioSession ses, int from, int to) {
+ assert from >= 0 && from < clientWorkers.size() : from;
+ assert to >= 0 && to < clientWorkers.size() : to;
+ assert from != to;
+
+ GridSelectorNioSessionImpl ses0 = (GridSelectorNioSessionImpl)ses;
+
+ SessionMoveFuture fut = new SessionMoveFuture(ses0, to);
+
+ if (!ses0.offerMove(clientWorkers.get(from), fut))
+ fut.onDone(false);
+ }
+
+ /**
* @param ses Session.
* @param op Operation.
* @return Future for operation.
*/
- GridNioFuture<?> pauseResumeReads(GridNioSession ses, NioOperation op) {
+ private GridNioFuture<?> pauseResumeReads(GridNioSession ses, NioOperation op) {
assert ses instanceof GridSelectorNioSessionImpl;
assert op == NioOperation.PAUSE_READ || op == NioOperation.RESUME_READ;
@@ -546,7 +683,7 @@ public class GridNioServer<T> {
NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, op);
- clientWorkers.get(impl.selectorIndex()).offer(fut);
+ impl.offerStateChange(fut);
return fut;
}
@@ -555,6 +692,9 @@ public class GridNioServer<T> {
*
*/
public void dumpStats() {
+ U.warn(log, "NIO server statistics [readerSesBalanceCnt=" + readerMoveCnt.get() +
+ ", writerSesBalanceCnt=" + writerMoveCnt.get() + ']');
+
for (int i = 0; i < clientWorkers.size(); i++)
clientWorkers.get(i).offer(new NioOperationFuture<Void>(null, NioOperation.DUMP_STATS));
}
@@ -675,12 +815,35 @@ public class GridNioServer<T> {
* @param req Request to balance.
*/
private synchronized void offerBalanced(NioOperationFuture req) {
- clientWorkers.get(balanceIdx).offer(req);
+ assert req.operation() == NioOperation.REGISTER : req;
+ assert req.socketChannel() != null : req;
+
+ int workers = clientWorkers.size();
+
+ int balanceIdx;
+
+ if (workers > 1) {
+ if (req.accepted()) {
+ balanceIdx = readBalanceIdx;
+
+ readBalanceIdx += 2;
+
+ if (readBalanceIdx >= workers)
+ readBalanceIdx = 0;
+ }
+ else {
+ balanceIdx = writeBalanceIdx;
- balanceIdx++;
+ writeBalanceIdx += 2;
- if (balanceIdx == clientWorkers.size())
+ if (writeBalanceIdx >= workers)
+ writeBalanceIdx = 1;
+ }
+ }
+ else
balanceIdx = 0;
+
+ clientWorkers.get(balanceIdx).offer(req);
}
/** {@inheritDoc} */
@@ -792,21 +955,30 @@ public class GridNioServer<T> {
while (true) {
ByteBuffer buf = ses.removeMeta(BUF_META_KEY);
- NioOperationFuture<?> req = ses.removeMeta(NIO_OPERATION.ordinal());
+ SessionWriteRequest req = ses.removeMeta(NIO_OPERATION.ordinal());
// Check if there were any pending data from previous writes.
if (buf == null) {
assert req == null;
- req = (NioOperationFuture<?>)ses.pollFuture();
+ req = ses.pollFuture();
if (req == null) {
- key.interestOps(key.interestOps() & (~SelectionKey.OP_WRITE));
+ if (ses.procWrite.get()) {
+ ses.procWrite.set(false);
+
+ if (ses.writeQueue().isEmpty()) {
+ if ((key.interestOps() & SelectionKey.OP_WRITE) != 0)
+ key.interestOps(key.interestOps() & (~SelectionKey.OP_WRITE));
+ }
+ else
+ ses.procWrite.set(true);
+ }
break;
}
- buf = req.message();
+ buf = (ByteBuffer)req.message();
}
if (!skipWrite) {
@@ -841,10 +1013,15 @@ public class GridNioServer<T> {
// Message was successfully written.
assert req != null;
- req.onDone();
+ req.onMessageWritten();
}
}
}
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(ByteBufferNioClientWorker.class, this, super.toString());
+ }
}
/**
@@ -909,6 +1086,7 @@ public class GridNioServer<T> {
metricsLsnr.onBytesReceived(cnt);
ses.bytesReceived(cnt);
+ onRead(cnt);
readBuf.flip();
@@ -921,6 +1099,12 @@ public class GridNioServer<T> {
readBuf.compact();
else
readBuf.clear();
+
+ if (ses.hasSystemMessage() && !ses.procWrite.get()) {
+ ses.procWrite.set(true);
+
+ registerWrite(ses);
+ }
}
catch (IgniteCheckedException e) {
close(ses, e);
@@ -993,16 +1177,29 @@ public class GridNioServer<T> {
if (ses.meta(WRITE_BUF_LIMIT) != null)
buf.limit((int)ses.meta(WRITE_BUF_LIMIT));
- NioOperationFuture<?> req = ses.removeMeta(NIO_OPERATION.ordinal());
+ SessionWriteRequest req = ses.removeMeta(NIO_OPERATION.ordinal());
while (true) {
if (req == null) {
- req = (NioOperationFuture<?>)ses.pollFuture();
+ req = systemMessage(ses);
- if (req == null && buf.position() == 0) {
- key.interestOps(key.interestOps() & (~SelectionKey.OP_WRITE));
+ if (req == null) {
+ req = ses.pollFuture();
- break;
+ if (req == null && buf.position() == 0) {
+ if (ses.procWrite.get()) {
+ ses.procWrite.set(false);
+
+ if (ses.writeQueue().isEmpty()) {
+ if ((key.interestOps() & SelectionKey.OP_WRITE) != 0)
+ key.interestOps(key.interestOps() & (~SelectionKey.OP_WRITE));
+ }
+ else
+ ses.procWrite.set(true);
+ }
+
+ break;
+ }
}
}
@@ -1010,7 +1207,7 @@ public class GridNioServer<T> {
boolean finished = false;
if (req != null) {
- msg = req.directMessage();
+ msg = (Message)req.message();
assert msg != null;
@@ -1025,14 +1222,17 @@ public class GridNioServer<T> {
// Fill up as many messages as possible to write buffer.
while (finished) {
- req.onDone();
+ req.onMessageWritten();
- req = (NioOperationFuture<?>)ses.pollFuture();
+ req = systemMessage(ses);
+
+ if (req == null)
+ req = ses.pollFuture();
if (req == null)
break;
- msg = req.directMessage();
+ msg = (Message)req.message();
assert msg != null;
@@ -1129,13 +1329,31 @@ public class GridNioServer<T> {
ses.bytesSent(cnt);
if (!buf.hasRemaining())
- queue.remove(buf);
+ queue.poll();
else
break;
}
}
/**
+ * @param ses Session.
+ * @return System message request.
+ */
+ private SessionWriteRequest systemMessage(GridSelectorNioSessionImpl ses) {
+ if (ses.hasSystemMessage()) {
+ Object msg = ses.systemMessage();
+
+ SessionWriteRequest req = new WriteRequestSystemImpl(ses, msg);
+
+ assert !ses.hasSystemMessage();
+
+ return req;
+ }
+
+ return null;
+ }
+
+ /**
* Processes write-ready event on the key.
*
* @param key Key that is ready to be written.
@@ -1147,7 +1365,7 @@ public class GridNioServer<T> {
GridSelectorNioSessionImpl ses = (GridSelectorNioSessionImpl)key.attachment();
ByteBuffer buf = ses.writeBuffer();
- NioOperationFuture<?> req = ses.removeMeta(NIO_OPERATION.ordinal());
+ SessionWriteRequest req = ses.removeMeta(NIO_OPERATION.ordinal());
MessageWriter writer = ses.meta(MSG_WRITER.ordinal());
@@ -1161,12 +1379,25 @@ public class GridNioServer<T> {
}
if (req == null) {
- req = (NioOperationFuture<?>)ses.pollFuture();
+ req = systemMessage(ses);
- if (req == null && buf.position() == 0) {
- key.interestOps(key.interestOps() & (~SelectionKey.OP_WRITE));
+ if (req == null) {
+ req = ses.pollFuture();
- return;
+ if (req == null && buf.position() == 0) {
+ if (ses.procWrite.get()) {
+ ses.procWrite.set(false);
+
+ if (ses.writeQueue().isEmpty()) {
+ if ((key.interestOps() & SelectionKey.OP_WRITE) != 0)
+ key.interestOps(key.interestOps() & (~SelectionKey.OP_WRITE));
+ }
+ else
+ ses.procWrite.set(true);
+ }
+
+ return;
+ }
}
}
@@ -1174,9 +1405,9 @@ public class GridNioServer<T> {
boolean finished = false;
if (req != null) {
- msg = req.directMessage();
+ msg = (Message)req.message();
- assert msg != null;
+ assert msg != null : req;
if (writer != null)
writer.setCurrentWriteClass(msg.getClass());
@@ -1189,14 +1420,17 @@ public class GridNioServer<T> {
// Fill up as many messages as possible to write buffer.
while (finished) {
- req.onDone();
+ req.onMessageWritten();
- req = (NioOperationFuture<?>)ses.pollFuture();
+ req = systemMessage(ses);
+
+ if (req == null)
+ req = ses.pollFuture();
if (req == null)
break;
- msg = req.directMessage();
+ msg = (Message)req.message();
assert msg != null;
@@ -1223,6 +1457,7 @@ public class GridNioServer<T> {
metricsLsnr.onBytesSent(cnt);
ses.bytesSent(cnt);
+ onWrite(cnt);
}
else {
// For test purposes only (skipWrite is set to true in tests only).
@@ -1242,14 +1477,19 @@ public class GridNioServer<T> {
else
buf.clear();
}
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(DirectNioClientWorker.class, this, super.toString());
+ }
}
/**
* Thread performing only read operations from the channel.
*/
- private abstract class AbstractNioClientWorker extends GridWorker {
+ private abstract class AbstractNioClientWorker extends GridWorker implements GridNioWorker {
/** Queue of change requests on this selector. */
- private final ConcurrentLinkedQueue<NioOperationFuture> changeReqs = new ConcurrentLinkedQueue<>();
+ private final ConcurrentLinkedQueue<SessionChangeRequest> changeReqs = new ConcurrentLinkedQueue<>();
/** Selector to select read events. */
private Selector selector;
@@ -1260,6 +1500,25 @@ public class GridNioServer<T> {
/** Worker index. */
private final int idx;
+ /** */
+ private long bytesRcvd;
+
+ /** */
+ private long bytesSent;
+
+ /** */
+ private volatile long bytesRcvd0;
+
+ /** */
+ private volatile long bytesSent0;
+
+ /** Sessions assigned to this worker. */
+ private final GridConcurrentHashSet<GridSelectorNioSessionImpl> workerSessions =
+ new GridConcurrentHashSet<>();
+
+ /** {@code True} if worker has called or is about to call {@code Selector.select()}. */
+ private volatile boolean select;
+
/**
* @param idx Index of this worker in server's array.
* @param gridName Grid name.
@@ -1322,15 +1581,15 @@ public class GridNioServer<T> {
try {
SelectedSelectionKeySet selectedKeySet = new SelectedSelectionKeySet();
- Class<?> selectorImplClass =
+ Class<?> selectorImplCls =
Class.forName("sun.nio.ch.SelectorImpl", false, U.gridClassLoader());
// Ensure the current selector implementation is what we can instrument.
- if (!selectorImplClass.isAssignableFrom(selector.getClass()))
+ if (!selectorImplCls.isAssignableFrom(selector.getClass()))
return;
- Field selectedKeysField = selectorImplClass.getDeclaredField("selectedKeys");
- Field publicSelectedKeysField = selectorImplClass.getDeclaredField("publicSelectedKeys");
+ Field selectedKeysField = selectorImplCls.getDeclaredField("selectedKeys");
+ Field publicSelectedKeysField = selectorImplCls.getDeclaredField("publicSelectedKeys");
selectedKeysField.setAccessible(true);
publicSelectedKeysField.setAccessible(true);
@@ -1357,48 +1616,126 @@ public class GridNioServer<T> {
*
* @param req Change request.
*/
- private void offer(NioOperationFuture req) {
+ public void offer(SessionChangeRequest req) {
changeReqs.offer(req);
+ if (select)
+ selector.wakeup();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void offer(Collection<SessionChangeRequest> reqs) {
+ for (SessionChangeRequest req : reqs)
+ changeReqs.offer(req);
+
selector.wakeup();
}
+ /** {@inheritDoc} */
+ @Override public List<SessionChangeRequest> clearSessionRequests(GridNioSession ses) {
+ List<SessionChangeRequest> sesReqs = null;
+
+ for (SessionChangeRequest changeReq : changeReqs) {
+ if (changeReq.session() == ses && !(changeReq instanceof SessionMoveFuture)) {
+ boolean rmv = changeReqs.remove(changeReq);
+
+ assert rmv : changeReq;
+
+ if (sesReqs == null)
+ sesReqs = new ArrayList<>();
+
+ sesReqs.add(changeReq);
+ }
+ }
+
+ return sesReqs;
+ }
+
/**
* Processes read and write events and registration requests.
*
* @throws IgniteCheckedException If IOException occurred or thread was unable to add worker to workers pool.
*/
@SuppressWarnings("unchecked")
- private void bodyInternal() throws IgniteCheckedException {
+ private void bodyInternal() throws IgniteCheckedException, InterruptedException {
try {
long lastIdleCheck = U.currentTimeMillis();
+ mainLoop:
while (!closed && selector.isOpen()) {
- NioOperationFuture req;
+ SessionChangeRequest req0;
- while ((req = changeReqs.poll()) != null) {
- switch (req.operation()) {
+ while ((req0 = changeReqs.poll()) != null) {
+ switch (req0.operation()) {
case REGISTER: {
- register(req);
+ register((NioOperationFuture)req0);
break;
}
- case REQUIRE_WRITE: {
- //Just register write key.
- SelectionKey key = req.session().key();
+ case MOVE: {
+ SessionMoveFuture f = (SessionMoveFuture)req0;
- if (key.isValid()) {
- key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
+ GridSelectorNioSessionImpl ses = f.session();
+
+ if (idx == f.toIdx) {
+ assert f.movedSocketChannel() != null : f;
+
+ boolean add = workerSessions.add(ses);
+
+ assert add;
- // Update timestamp to protected against false write timeout.
- ((GridNioSessionImpl)key.attachment()).bytesSent(0);
+ ses.finishMoveSession(this);
+
+ if (idx % 2 == 0)
+ readerMoveCnt.incrementAndGet();
+ else
+ writerMoveCnt.incrementAndGet();
+
+ SelectionKey key = f.movedSocketChannel().register(selector,
+ SelectionKey.OP_READ | SelectionKey.OP_WRITE,
+ ses);
+
+ ses.key(key);
+
+ ses.procWrite.set(true);
+
+ f.onDone(true);
+ }
+ else {
+ assert f.movedSocketChannel() == null : f;
+
+ if (workerSessions.remove(ses)) {
+ ses.startMoveSession(this);
+
+ SelectionKey key = ses.key();
+
+ assert key.channel() != null : key;
+
+ f.movedSocketChannel((SocketChannel)key.channel());
+
+ key.cancel();
+
+ clientWorkers.get(f.toIndex()).offer(f);
+ }
+ else
+ f.onDone(false);
}
break;
}
+ case REQUIRE_WRITE: {
+ SessionWriteRequest req = (SessionWriteRequest)req0;
+
+ registerWrite((GridSelectorNioSessionImpl)req.session());
+
+ break;
+ }
+
case CLOSE: {
+ NioOperationFuture req = (NioOperationFuture)req0;
+
if (close(req.session(), null))
req.onDone(true);
else
@@ -1408,6 +1745,8 @@ public class GridNioServer<T> {
}
case PAUSE_READ: {
+ NioOperationFuture req = (NioOperationFuture)req0;
+
SelectionKey key = req.session().key();
if (key.isValid()) {
@@ -1426,6 +1765,8 @@ public class GridNioServer<T> {
}
case RESUME_READ: {
+ NioOperationFuture req = (NioOperationFuture)req0;
+
SelectionKey key = req.session().key();
if (key.isValid()) {
@@ -1444,76 +1785,66 @@ public class GridNioServer<T> {
}
case DUMP_STATS: {
- StringBuilder sb = new StringBuilder();
+ NioOperationFuture req = (NioOperationFuture)req0;
- Set<SelectionKey> keys = selector.keys();
-
- sb.append(U.nl())
- .append(">> Selector info [idx=").append(idx)
- .append(", keysCnt=").append(keys.size())
- .append("]").append(U.nl());
-
- for (SelectionKey key : keys) {
- GridSelectorNioSessionImpl ses = (GridSelectorNioSessionImpl)key.attachment();
-
- MessageWriter writer = ses.meta(MSG_WRITER.ordinal());
- MessageReader reader = ses.meta(GridDirectParser.READER_META_KEY);
-
- sb.append(" Connection info [")
- .append("rmtAddr=").append(ses.remoteAddress())
- .append(", locAddr=").append(ses.localAddress());
-
- GridNioRecoveryDescriptor desc = ses.recoveryDescriptor();
+ try {
+ dumpStats();
+ }
+ finally {
+ // Complete the request just in case (none should wait on this future).
+ req.onDone(true);
+ }
+ }
+ }
+ }
- if (desc != null) {
- sb.append(", msgsSent=").append(desc.sent())
- .append(", msgsAckedByRmt=").append(desc.acked())
- .append(", msgsRcvd=").append(desc.received())
- .append(", descIdHash=").append(System.identityHashCode(desc));
- }
- else
- sb.append(", recoveryDesc=null");
+ int res = 0;
- sb.append(", bytesRcvd=").append(ses.bytesReceived())
- .append(", bytesSent=").append(ses.bytesSent())
- .append(", opQueueSize=").append(ses.writeQueueSize())
- .append(", msgWriter=").append(writer != null ? writer.toString() : "null")
- .append(", msgReader=").append(reader != null ? reader.toString() : "null");
+ for (long i = 0; i < selectorSpins && res == 0; i++) {
+ res = selector.selectNow();
- int cnt = 0;
+ if (res > 0) {
+ // Walk through the ready keys collection and process network events.
+ if (selectedKeys == null)
+ processSelectedKeys(selector.selectedKeys());
+ else
+ processSelectedKeysOptimized(selectedKeys.flip());
+ }
- for (GridNioFuture<?> fut : ses.writeQueue()) {
- if (cnt == 0)
- sb.append(",\n opQueue=[").append(fut);
- else
- sb.append(',').append(fut);
+ if (!changeReqs.isEmpty())
+ continue mainLoop;
- if (++cnt == 5) {
- sb.append(']');
+ // Just in case we do busy selects.
+ long now = U.currentTimeMillis();
- break;
- }
- }
+ if (now - lastIdleCheck > 2000) {
+ lastIdleCheck = now;
+ checkIdle(selector.keys());
+ }
- sb.append("]").append(U.nl());
- }
+ if (isCancelled())
+ return;
+ }
- U.warn(log, sb.toString());
+ // Falling to blocking select.
+ select = true;
- // Complete the request just in case (none should wait on this future).
- req.onDone(true);
- }
+ try {
+ if (!changeReqs.isEmpty())
+ continue;
+
+ // Wake up every 2 seconds to check if closed.
+ if (selector.select(2000) > 0) {
+ // Walk through the ready keys collection and process network events.
+ if (selectedKeys == null)
+ processSelectedKeys(selector.selectedKeys());
+ else
+ processSelectedKeysOptimized(selectedKeys.flip());
}
}
-
- // Wake up every 2 seconds to check if closed.
- if (selector.select(2000) > 0) {
- // Walk through the ready keys collection and process network events.
- if (selectedKeys == null)
- processSelectedKeys(selector.selectedKeys());
- else
- processSelectedKeysOptimized(selectedKeys.flip());
+ finally {
+ select = false;
}
long now = U.currentTimeMillis();
@@ -1554,6 +1885,98 @@ public class GridNioServer<T> {
}
/**
+ * @param ses Session.
+ */
+ public final void registerWrite(GridSelectorNioSessionImpl ses) {
+ SelectionKey key = ses.key();
+
+ if (key.isValid()) {
+ if ((key.interestOps() & SelectionKey.OP_WRITE) == 0)
+ key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
+
+ // Update timestamp to protected against false write timeout.
+ ses.bytesSent(0);
+ }
+ }
+
+ /**
+ *
+ */
+ private void dumpStats() {
+ StringBuilder sb = new StringBuilder();
+
+ Set<SelectionKey> keys = selector.keys();
+
+ sb.append(U.nl())
+ .append(">> Selector info [idx=").append(idx)
+ .append(", keysCnt=").append(keys.size())
+ .append(", bytesRcvd=").append(bytesRcvd)
+ .append(", bytesRcvd0=").append(bytesRcvd0)
+ .append(", bytesSent=").append(bytesSent)
+ .append(", bytesSent0=").append(bytesSent0)
+ .append("]").append(U.nl());
+
+ for (SelectionKey key : keys) {
+ GridSelectorNioSessionImpl ses = (GridSelectorNioSessionImpl)key.attachment();
+
+ MessageWriter writer = ses.meta(MSG_WRITER.ordinal());
+ MessageReader reader = ses.meta(GridDirectParser.READER_META_KEY);
+
+ sb.append(" Connection info [")
+ .append("in=").append(ses.accepted())
+ .append(", rmtAddr=").append(ses.remoteAddress())
+ .append(", locAddr=").append(ses.localAddress());
+
+ GridNioRecoveryDescriptor outDesc = ses.outRecoveryDescriptor();
+
+ if (outDesc != null) {
+ sb.append(", msgsSent=").append(outDesc.sent())
+ .append(", msgsAckedByRmt=").append(outDesc.acked())
+ .append(", descIdHash=").append(System.identityHashCode(outDesc));
+ }
+ else
+ sb.append(", outRecoveryDesc=null");
+
+ GridNioRecoveryDescriptor inDesc = ses.inRecoveryDescriptor();
+
+ if (inDesc != null) {
+ sb.append(", msgsRcvd=").append(inDesc.received())
+ .append(", lastAcked=").append(inDesc.lastAcknowledged())
+ .append(", descIdHash=").append(System.identityHashCode(inDesc));
+ }
+ else
+ sb.append(", inRecoveryDesc=null");
+
+ sb.append(", bytesRcvd=").append(ses.bytesReceived())
+ .append(", bytesRcvd0=").append(ses.bytesReceived0())
+ .append(", bytesSent=").append(ses.bytesSent())
+ .append(", bytesSent0=").append(ses.bytesSent0())
+ .append(", opQueueSize=").append(ses.writeQueueSize())
+ .append(", msgWriter=").append(writer != null ? writer.toString() : "null")
+ .append(", msgReader=").append(reader != null ? reader.toString() : "null");
+
+ int cnt = 0;
+
+ for (SessionWriteRequest req : ses.writeQueue()) {
+ if (cnt == 0)
+ sb.append(",\n opQueue=[").append(req);
+ else
+ sb.append(',').append(req);
+
+ if (++cnt == 5) {
+ sb.append(']');
+
+ break;
+ }
+ }
+
+ sb.append("]").append(U.nl());
+ }
+
+ U.warn(log, sb.toString());
+ }
+
+ /**
* Processes keys selected by a selector.
*
* @param keys Selected keys.
@@ -1671,7 +2094,9 @@ public class GridNioServer<T> {
long idleTimeout0 = idleTimeout;
- if (!opWrite && now - ses.lastReceiveTime() > idleTimeout0 && now - ses.lastSendScheduleTime() > idleTimeout0) {
+ if (!opWrite &&
+ now - ses.lastReceiveTime() > idleTimeout0 &&
+ now - ses.lastSendScheduleTime() > idleTimeout0) {
filterChain.onSessionIdleTimeout(ses);
// Update timestamp to avoid multiple notifications within one timeout interval.
@@ -1715,7 +2140,7 @@ public class GridNioServer<T> {
final GridSelectorNioSessionImpl ses = new GridSelectorNioSessionImpl(
log,
- idx,
+ this,
filterChain,
(InetSocketAddress)sockCh.getLocalAddress(),
(InetSocketAddress)sockCh.getRemoteAddress(),
@@ -1739,6 +2164,7 @@ public class GridNioServer<T> {
resend(ses);
sessions.add(ses);
+ workerSessions.add(ses);
try {
filterChain.onSessionOpened(ses);
@@ -1764,7 +2190,7 @@ public class GridNioServer<T> {
}
/**
- * Closes the ses and all associated resources, then notifies the listener.
+ * Closes the session and all associated resources, then notifies the listener.
*
* @param ses Session to be closed.
* @param e Exception to be passed to the listener, if any.
@@ -1781,12 +2207,10 @@ public class GridNioServer<T> {
}
sessions.remove(ses);
+ workerSessions.remove(ses);
SelectionKey key = ses.key();
- // Shutdown input and output so that remote client will see correct socket close.
- Socket sock = ((SocketChannel)key.channel()).socket();
-
if (ses.setClosed()) {
ses.onClosed();
@@ -1798,6 +2222,9 @@ public class GridNioServer<T> {
((DirectBuffer)ses.readBuffer()).cleaner().clean();
}
+ // Shutdown input and output so that remote client will see correct socket close.
+ Socket sock = ((SocketChannel)key.channel()).socket();
+
try {
try {
sock.shutdownInput();
@@ -1824,28 +2251,35 @@ public class GridNioServer<T> {
ses.removeMeta(BUF_META_KEY);
// Since ses is in closed state, no write requests will be added.
- NioOperationFuture<?> fut = ses.removeMeta(NIO_OPERATION.ordinal());
+ SessionWriteRequest req = ses.removeMeta(NIO_OPERATION.ordinal());
+
+ GridNioRecoveryDescriptor outRecovery = ses.outRecoveryDescriptor();
+ GridNioRecoveryDescriptor inRecovery = ses.inRecoveryDescriptor();
- GridNioRecoveryDescriptor recovery = ses.recoveryDescriptor();
+ IOException err = new IOException("Failed to send message (connection was closed): " + ses);
- if (recovery != null) {
+ if (outRecovery != null || inRecovery != null) {
try {
// Poll will update recovery data.
- while ((fut = (NioOperationFuture<?>)ses.pollFuture()) != null) {
- if (fut.skipRecovery())
- fut.connectionClosed();
+ while ((req = ses.pollFuture()) != null) {
+ if (req.skipRecovery())
+ req.onError(err);
}
}
finally {
- recovery.release();
+ if (outRecovery != null)
+ outRecovery.release();
+
+ if (inRecovery != null && inRecovery != outRecovery)
+ inRecovery.release();
}
}
else {
- if (fut != null)
- fut.connectionClosed();
+ if (req != null)
+ req.onError(err);
- while ((fut = (NioOperationFuture<?>)ses.pollFuture()) != null)
- fut.connectionClosed();
+ while ((req = ses.pollFuture()) != null)
+ req.onError(err);
}
try {
@@ -1876,12 +2310,44 @@ public class GridNioServer<T> {
* @throws IOException If write failed.
*/
protected abstract void processWrite(SelectionKey key) throws IOException;
- }
- /**
- * Gets outbound messages queue size.
- *
- * @return Write queue size.
+ /**
+ * @param cnt
+ */
+ final void onRead(int cnt) {
+ bytesRcvd += cnt;
+ bytesRcvd0 += cnt;
+ }
+
+ /**
+ * @param cnt
+ */
+ final void onWrite(int cnt) {
+ bytesSent += cnt;
+ bytesSent0 += cnt;
+ }
+
+ /**
+ *
+ */
+ final void reset0() {
+ bytesSent0 = 0;
+ bytesRcvd0 = 0;
+
+ for (GridSelectorNioSessionImpl ses : workerSessions)
+ ses.reset0();
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(AbstractNioClientWorker.class, this, super.toString());
+ }
+ }
+
+ /**
+ * Gets outbound messages queue size.
+ *
+ * @return Write queue size.
*/
public int outboundMessagesQueueSize() {
int res = 0;
@@ -1952,6 +2418,9 @@ public class GridNioServer<T> {
if (selector.select(2000) > 0)
// Walk through the ready keys collection and process date requests.
processSelectedKeys(selector.selectedKeys());
+
+ if (balancer != null)
+ balancer.run();
}
}
// Ignore this exception as thread interruption is equal to 'close' call.
@@ -2048,10 +2517,13 @@ public class GridNioServer<T> {
/**
* Asynchronous operation that may be requested on selector.
*/
- private enum NioOperation {
+ enum NioOperation {
/** Register read key selection. */
REGISTER,
+ /** Move session between workers. */
+ MOVE,
+
/** Register write key selection. */
REQUIRE_WRITE,
@@ -2069,9 +2541,193 @@ public class GridNioServer<T> {
}
/**
+ *
+ */
+ private static final class WriteRequestSystemImpl implements SessionWriteRequest, SessionChangeRequest {
+ /** */
+ private final Object msg;
+
+ /** */
+ private final GridNioSession ses;
+
+ /**
+ * @param ses Session.
+ * @param msg Message.
+ */
+ WriteRequestSystemImpl(GridNioSession ses, Object msg) {
+ this.ses = ses;
+ this.msg = msg;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void messageThread(boolean msgThread) {
+ // No-op.
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean messageThread() {
+ return true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean skipRecovery() {
+ return true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void ackClosure(IgniteInClosure<IgniteException> c) {
+ throw new UnsupportedOperationException();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onAckReceived() {
+ throw new UnsupportedOperationException();
+ }
+
+ /** {@inheritDoc} */
+ @Override public IgniteInClosure<IgniteException> ackClosure() {
+ return null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onError(Exception e) {
+ // No-op.
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object message() {
+ return msg;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onMessageWritten() {
+ // No-op.
+ }
+
+ /** {@inheritDoc} */
+ @Override public void resetSession(GridNioSession ses) {
+ throw new UnsupportedOperationException();
+ }
+
+ /** {@inheritDoc} */
+ @Override public GridNioSession session() {
+ return ses;
+ }
+
+ /** {@inheritDoc} */
+ @Override public NioOperation operation() {
+ return NioOperation.REQUIRE_WRITE;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(WriteRequestSystemImpl.class, this);
+ }
+ }
+
+ /**
+ *
+ */
+ private static final class WriteRequestImpl implements SessionWriteRequest, SessionChangeRequest {
+ /** */
+ private GridNioSession ses;
+
+ /** */
+ private final Object msg;
+
+ /** */
+ private boolean msgThread;
+
+ /** */
+ private final boolean skipRecovery;
+
+ /** */
+ private IgniteInClosure<IgniteException> ackC;
+
+ /**
+ * @param ses Session.
+ * @param msg Message.
+ * @param skipRecovery Skip recovery flag.
+ */
+ WriteRequestImpl(GridNioSession ses, Object msg, boolean skipRecovery) {
+ this.ses = ses;
+ this.msg = msg;
+ this.skipRecovery = skipRecovery;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void messageThread(boolean msgThread) {
+ this.msgThread = msgThread;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean messageThread() {
+ return msgThread;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean skipRecovery() {
+ return skipRecovery;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void ackClosure(IgniteInClosure<IgniteException> c) {
+ ackC = c;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onAckReceived() {
+ assert msg instanceof Message;
+
+ ((Message)msg).onAckReceived();
+ }
+
+ /** {@inheritDoc} */
+ @Override public IgniteInClosure<IgniteException> ackClosure() {
+ return ackC;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onError(Exception e) {
+ // No-op.
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object message() {
+ return msg;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onMessageWritten() {
+ // No-op.
+ }
+
+ /** {@inheritDoc} */
+ @Override public void resetSession(GridNioSession ses) {
+ this.ses = ses;
+ }
+
+ /** {@inheritDoc} */
+ @Override public GridNioSession session() {
+ return ses;
+ }
+
+ /** {@inheritDoc} */
+ @Override public NioOperation operation() {
+ return NioOperation.REQUIRE_WRITE;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(WriteRequestImpl.class, this);
+ }
+ }
+
+ /**
* Class for requesting write and session close operations.
*/
- private static class NioOperationFuture<R> extends GridNioFutureImpl<R> {
+ private static class NioOperationFuture<R> extends GridNioFutureImpl<R> implements SessionWriteRequest,
+ SessionChangeRequest {
/** */
private static final long serialVersionUID = 0L;
@@ -2087,11 +2743,7 @@ public class GridNioServer<T> {
private NioOperation op;
/** Message. */
- @GridToStringExclude
- private ByteBuffer msg;
-
- /** Direct message. */
- private Message commMsg;
+ private Object msg;
/** */
@GridToStringExclude
@@ -2153,8 +2805,7 @@ public class GridNioServer<T> {
* @param op Requested operation.
* @param msg Message.
*/
- NioOperationFuture(GridSelectorNioSessionImpl ses, NioOperation op,
- ByteBuffer msg) {
+ NioOperationFuture(GridSelectorNioSessionImpl ses, NioOperation op, Object msg) {
assert ses != null;
assert op != null;
assert op != NioOperation.REGISTER;
@@ -2182,51 +2833,36 @@ public class GridNioServer<T> {
this.ses = ses;
this.op = op;
- this.commMsg = commMsg;
+ this.msg = commMsg;
this.skipRecovery = skipRecovery;
}
- /**
- * @return Requested change operation.
- */
- private NioOperation operation() {
+ /** {@inheritDoc} */
+ public NioOperation operation() {
return op;
}
- /**
- * @return Message.
- */
- private ByteBuffer message() {
+ /** {@inheritDoc} */
+ public Object message() {
return msg;
}
- /**
- * @return Direct message.
- */
- private Message directMessage() {
- return commMsg;
- }
-
- /**
- * @param ses New session instance.
- */
- private void resetSession(GridSelectorNioSessionImpl ses) {
- assert commMsg != null;
+ /** {@inheritDoc} */
+ public void resetSession(GridNioSession ses) {
+ assert msg instanceof Message : msg;
- this.ses = ses;
+ this.ses = (GridSelectorNioSessionImpl)ses;
}
/**
* @return Socket channel for register request.
*/
- private SocketChannel socketChannel() {
+ SocketChannel socketChannel() {
return sockCh;
}
- /**
- * @return Session for this change request.
- */
- private GridSelectorNioSessionImpl session() {
+ /** {@inheritDoc} */
+ public GridSelectorNioSessionImpl session() {
return ses;
}
@@ -2244,21 +2880,21 @@ public class GridNioServer<T> {
return meta;
}
- /**
- * Applicable to write futures only. Fails future with corresponding IOException.
- */
- private void connectionClosed() {
- assert op == NioOperation.REQUIRE_WRITE;
- assert ses != null;
-
- onDone(new IOException("Failed to send message (connection was closed): " + ses));
+ /** {@inheritDoc} */
+ @Override public void onError(Exception e) {
+ onDone(e);
}
/** {@inheritDoc} */
@Override public void onAckReceived() {
- assert commMsg != null;
+ assert msg instanceof Message : msg;
- commMsg.onAckReceived();
+ ((Message)msg).onAckReceived();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onMessageWritten() {
+ onDone();
}
/** {@inheritDoc} */
@@ -2273,6 +2909,59 @@ public class GridNioServer<T> {
}
/**
+ *
+ */
+ private static class SessionMoveFuture extends NioOperationFuture<Boolean> {
+ /** */
+ private final int toIdx;
+
+ /** */
+ @GridToStringExclude
+ private SocketChannel movedSockCh;
+
+ /**
+ * @param ses Session.
+ * @param toIdx Target worker index.
+ */
+ SessionMoveFuture(
+ GridSelectorNioSessionImpl ses,
+ int toIdx
+ ) {
+ super(ses, NioOperation.MOVE);
+
+ this.toIdx = toIdx;
+ }
+
+ /**
+ * @return Target worker index.
+ */
+ int toIndex() {
+ return toIdx;
+ }
+
+ /**
+ * @return Moved session socket channel.
+ */
+ SocketChannel movedSocketChannel() {
+ return movedSockCh;
+ }
+
+ /**
+ * @param movedSockCh Moved session socket channel.
+ */
+ void movedSocketChannel(SocketChannel movedSockCh) {
+ assert movedSockCh != null;
+
+ this.movedSockCh = movedSockCh;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(SessionMoveFuture.class, this, super.toString());
+ }
+ }
+
+ /**
* Filter forwarding messages from chain's head to this server.
*/
private class HeadFilter extends GridNioFilterAdapter {
@@ -2302,7 +2991,7 @@ public class GridNioServer<T> {
}
/** {@inheritDoc} */
- @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) {
+ @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) throws IgniteCheckedException {
if (directMode) {
boolean sslSys = sslFilter != null && msg instanceof ByteBuffer;
@@ -2313,18 +3002,18 @@ public class GridNioServer<T> {
queue.offer((ByteBuffer)msg);
- SelectionKey key = ((GridSelectorNioSessionImpl)ses).key();
+ GridSelectorNioSessionImpl ses0 = (GridSelectorNioSessionImpl)ses;
- if (key.isValid())
- key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
+ if (!ses0.procWrite.get() && ses0.procWrite.compareAndSet(false, true))
+ ses0.worker().registerWrite(ses0);
return null;
}
else
- return send(ses, (Message)msg);
+ return send(ses, (Message)msg, fut);
}
else
- return send(ses, (ByteBuffer)msg);
+ return send(ses, (ByteBuffer)msg, fut);
}
/** {@inheritDoc} */
@@ -2429,6 +3118,15 @@ public class GridNioServer<T> {
/** Message queue size listener. */
private IgniteBiInClosure<GridNioSession, Integer> msgQueueLsnr;
+ /** Name for threads identification. */
+ private String srvName;
+
+ /** */
+ private long selectorSpins;
+
+ /** NIO sessions balancing flag. */
+ private boolean balancing;
+
/**
* Finishes building the instance.
*
@@ -2442,6 +3140,8 @@ public class GridNioServer<T> {
log,
selectorCnt,
gridName,
+ srvName,
+ selectorSpins,
tcpNoDelay,
directBuf,
byteOrder,
@@ -2455,6 +3155,7 @@ public class GridNioServer<T> {
writerFactory,
skipRecoveryPred,
msgQueueLsnr,
+ balancing,
filters != null ? Arrays.copyOf(filters, filters.length) : EMPTY_FILTERS
);
@@ -2468,6 +3169,16 @@ public class GridNioServer<T> {
}
/**
+ * @param balancing NIO sessions balancing flag.
+ * @return This for chaining.
+ */
+ public Builder<T> balancing(boolean balancing) {
+ this.balancing = balancing;
+
+ return this;
+ }
+
+ /**
* @param addr Local address.
* @return This for chaining.
*/
@@ -2519,6 +3230,28 @@ public class GridNioServer<T> {
}
/**
+ * @param srvName Logical server name for threads identification.
+ * @return This for chaining.
+ */
+ public Builder<T> serverName(@Nullable String srvName) {
+ this.srvName = srvName;
+
+ return this;
+ }
+
+ /**
+ * @param selectorSpins Defines how many non-blocking {@code selector.selectNow()} should be made before
+ * falling into {@code selector.select(long)} in NIO server. Long value. Default is {@code 0}.
+ * Can be set to {@code Long.MAX_VALUE} so selector threads will never block.
+ * @return This for chaining.
+ */
+ public Builder<T> selectorSpins(long selectorSpins) {
+ this.selectorSpins = selectorSpins;
+
+ return this;
+ }
+
+ /**
* @param tcpNoDelay If TCP_NODELAY option should be set to accepted sockets.
* @return This for chaining.
*/
@@ -2678,4 +3411,225 @@ public class GridNioServer<T> {
return this;
}
}
+
+ /**
+ *
+ */
+ private class SizeBasedBalancer implements IgniteRunnable {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ private long lastBalance;
+
+ /** */
+ private final long balancePeriod;
+
+ /**
+ * @param balancePeriod Period.
+ */
+ SizeBasedBalancer(long balancePeriod) {
+ this.balancePeriod = balancePeriod;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void run() {
+ long now = U.currentTimeMillis();
+
+ if (lastBalance + balancePeriod < now) {
+ lastBalance = now;
+
+ long maxRcvd0 = -1, minRcvd0 = -1, maxSent0 = -1, minSent0 = -1;
+ int maxRcvdIdx = -1, minRcvdIdx = -1, maxSentIdx = -1, minSentIdx = -1;
+
+ for (int i = 0; i < clientWorkers.size(); i++) {
+ GridNioServer.AbstractNioClientWorker worker = clientWorkers.get(i);
+
+ int sesCnt = worker.workerSessions.size();
+
+ if (i % 2 == 0) {
+ // Reader.
+ long bytesRcvd0 = worker.bytesRcvd0;
+
+ if ((maxRcvd0 == -1 || bytesRcvd0 > maxRcvd0) && bytesRcvd0 > 0 && sesCnt > 1) {
+ maxRcvd0 = bytesRcvd0;
+ maxRcvdIdx = i;
+ }
+
+ if (minRcvd0 == -1 || bytesRcvd0 < minRcvd0) {
+ minRcvd0 = bytesRcvd0;
+ minRcvdIdx = i;
+ }
+ }
+ else {
+ // Writer.
+ long bytesSent0 = worker.bytesSent0;
+
+ if ((maxSent0 == -1 || bytesSent0 > maxSent0) && bytesSent0 > 0 && sesCnt > 1) {
+ maxSent0 = bytesSent0;
+ maxSentIdx = i;
+ }
+
+ if (minSent0 == -1 || bytesSent0 < minSent0) {
+ minSent0 = bytesSent0;
+ minSentIdx = i;
+ }
+ }
+ }
+
+ if (log.isDebugEnabled())
+ log.debug("Balancing data [minSent0=" + minSent0 + ", minSentIdx=" + minSentIdx +
+ ", maxSent0=" + maxSent0 + ", maxSentIdx=" + maxSentIdx +
+ ", minRcvd0=" + minRcvd0 + ", minRcvdIdx=" + minRcvdIdx +
+ ", maxRcvd0=" + maxRcvd0 + ", maxRcvdIdx=" + maxRcvdIdx + ']');
+
+ if (maxSent0 != -1 && minSent0 != -1) {
+ GridSelectorNioSessionImpl ses = null;
+
+ long sentDiff = maxSent0 - minSent0;
+ long delta = sentDiff;
+ double threshold = sentDiff * 0.9;
+
+ GridConcurrentHashSet<GridSelectorNioSessionImpl> sessions =
+ clientWorkers.get(maxSentIdx).workerSessions;
+
+ for (GridSelectorNioSessionImpl ses0 : sessions) {
+ long bytesSent0 = ses0.bytesSent0();
+
+ if (bytesSent0 < threshold &&
+ (ses == null || delta > U.safeAbs(bytesSent0 - sentDiff / 2))) {
+ ses = ses0;
+ delta = U.safeAbs(bytesSent0 - sentDiff / 2);
+ }
+ }
+
+ if (ses != null) {
+ if (log.isDebugEnabled())
+ log.debug("Will move session to less loaded writer [ses=" + ses +
+ ", from=" + maxSentIdx + ", to=" + minSentIdx + ']');
+
+ moveSession(ses, maxSentIdx, minSentIdx);
+ }
+ else {
+ if (log.isDebugEnabled())
+ log.debug("Unable to find session to move for writers.");
+ }
+ }
+
+ if (maxRcvd0 != -1 && minRcvd0 != -1) {
+ GridSelectorNioSessionImpl ses = null;
+
+ long rcvdDiff = maxRcvd0 - minRcvd0;
+ long delta = rcvdDiff;
+ double threshold = rcvdDiff * 0.9;
+
+ GridConcurrentHashSet<GridSelectorNioSessionImpl> sessions =
+ clientWorkers.get(maxRcvdIdx).workerSessions;
+
+ for (GridSelectorNioSessionImpl ses0 : sessions) {
+ long bytesRcvd0 = ses0.bytesReceived0();
+
+ if (bytesRcvd0 < threshold &&
+ (ses == null || delta > U.safeAbs(bytesRcvd0 - rcvdDiff / 2))) {
+ ses = ses0;
+ delta = U.safeAbs(bytesRcvd0 - rcvdDiff / 2);
+ }
+ }
+
+ if (ses != null) {
+ if (log.isDebugEnabled())
+ log.debug("Will move session to less loaded reader [ses=" + ses +
+ ", from=" + maxRcvdIdx + ", to=" + minRcvdIdx + ']');
+
+ moveSession(ses, maxRcvdIdx, minRcvdIdx);
+ }
+ else {
+ if (log.isDebugEnabled())
+ log.debug("Unable to find session to move for readers.");
+ }
+ }
+
+ for (int i = 0; i < clientWorkers.size(); i++) {
+ GridNioServer.AbstractNioClientWorker worker = clientWorkers.get(i);
+
+ worker.reset0();
+ }
+ }
+ }
+ }
+
+ /**
+ * For tests only.
+ */
+ @SuppressWarnings("unchecked")
+ private class RandomBalancer implements IgniteRunnable {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** {@inheritDoc} */
+ @Override public void run() {
+ ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+ int w1 = rnd.nextInt(clientWorkers.size());
+
+ if (clientWorkers.get(w1).workerSessions.isEmpty())
+ return;
+
+ int w2 = rnd.nextInt(clientWorkers.size());
+
+ while (w2 == w1)
+ w2 = rnd.nextInt(clientWorkers.size());
+
+ GridNioSession ses = randomSession(clientWorkers.get(w1));
+
+ if (ses != null) {
+ log.info("Move session [from=" + w1 +
+ ", to=" + w2 +
+ ", ses=" + ses + ']');
+
+ moveSession(ses, w1, w2);
+ }
+ }
+
+ /**
+ * @param worker Worker.
+ * @return NIO session.
+ */
+ private GridNioSession randomSession(GridNioServer.AbstractNioClientWorker worker) {
+ Collection<GridNioSession> sessions = worker.workerSessions;
+
+ int size = sessions.size();
+
+ if (size == 0)
+ return null;
+
+ int idx = ThreadLocalRandom.current().nextInt(size);
+
+ Iterator<GridNioSession> it = sessions.iterator();
+
+ int cnt = 0;
+
+ while (it.hasNext()) {
+ GridNioSession ses = it.next();
+
+ if (cnt == idx)
+ return ses;
+ }
+
+ return null;
+ }
+
+ }
+
+ /**
+ *
+ */
+ interface SessionChangeRequest {
+ GridNioSession session();
+
+ /**
+ * @return Requested change operation.
+ */
+ NioOperation operation();
+ }
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java
index e4a7225..c1b60ab 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java
@@ -18,6 +18,7 @@
package org.apache.ignite.internal.util.nio;
import java.net.InetSocketAddress;
+import org.apache.ignite.IgniteCheckedException;
import org.jetbrains.annotations.Nullable;
/**
@@ -105,6 +106,11 @@ public interface GridNioSession {
public GridNioFuture<?> send(Object msg);
/**
+ * @param msg Message to be sent.
+ */
+ public void sendNoFuture(Object msg) throws IgniteCheckedException;
+
+ /**
* Gets metadata associated with specified key.
*
* @param key Key to look up.
@@ -158,10 +164,25 @@ public interface GridNioSession {
/**
* @param recoveryDesc Recovery descriptor.
*/
- public void recoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc);
+ public void outRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc);
+
+ /**
+ * @param recoveryDesc Recovery descriptor.
+ */
+ public void inRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc);
/**
* @return Recovery descriptor if recovery is supported, {@code null otherwise.}
*/
- @Nullable public GridNioRecoveryDescriptor recoveryDescriptor();
+ @Nullable public GridNioRecoveryDescriptor outRecoveryDescriptor();
+
+ /**
+ * @return Recovery descriptor if recovery is supported, {@code null otherwise.}
+ */
+ @Nullable public GridNioRecoveryDescriptor inRecoveryDescriptor();
+
+ /**
+ * @param msg System message to send.
+ */
+ public void systemMessage(Object msg);
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java
index 0bcfe64..7424531 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java
@@ -51,6 +51,12 @@ public class GridNioSessionImpl implements GridNioSession {
/** Received bytes counter. */
private volatile long bytesRcvd;
+ /** Sent bytes since last NIO sessions balancing. */
+ private volatile long bytesSent0;
+
+ /** Received bytes since last NIO sessions balancing. */
+ private volatile long bytesRcvd0;
+
/** Last send schedule timestamp. */
private volatile long sndSchedTime;
@@ -99,7 +105,7 @@ public class GridNioSessionImpl implements GridNioSession {
try {
resetSendScheduleTime();
- return chain().onSessionWrite(this, msg);
+ return chain().onSessionWrite(this, msg, true);
}
catch (IgniteCheckedException e) {
close();
@@ -109,6 +115,18 @@ public class GridNioSessionImpl implements GridNioSession {
}
/** {@inheritDoc} */
+ @Override public void sendNoFuture(Object msg) throws IgniteCheckedException {
+ try {
+ chain().onSessionWrite(this, msg, false);
+ }
+ catch (IgniteCheckedException e) {
+ close();
+
+ throw e;
+ }
+ }
+
+ /** {@inheritDoc} */
@Override public GridNioFuture<?> resumeReads() {
try {
return chain().onResumeReads(this);
@@ -163,6 +181,28 @@ public class GridNioSessionImpl implements GridNioSession {
return bytesRcvd;
}
+ /**
+ * @return Sent bytes since last NIO sessions balancing.
+ */
+ public long bytesSent0() {
+ return bytesSent0;
+ }
+
+ /**
+ * @return Received bytes since last NIO sessions balancing.
+ */
+ public long bytesReceived0() {
+ return bytesRcvd0;
+ }
+
+ /**
+ *
+ */
+ public void reset0() {
+ bytesSent0 = 0;
+ bytesRcvd0 = 0;
+ }
+
/** {@inheritDoc} */
@Override public long createTime() {
return createTime;
@@ -240,6 +280,7 @@ public class GridNioSessionImpl implements GridNioSession {
*/
public void bytesSent(int cnt) {
bytesSent += cnt;
+ bytesSent0 += cnt;
lastSndTime = U.currentTimeMillis();
}
@@ -253,6 +294,7 @@ public class GridNioSessionImpl implements GridNioSession {
*/
public void bytesReceived(int cnt) {
bytesRcvd += cnt;
+ bytesRcvd0 += cnt;
lastRcvTime = U.currentTimeMillis();
}
@@ -296,17 +338,32 @@ public class GridNioSessionImpl implements GridNioSession {
}
/** {@inheritDoc} */
- @Override public void recoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+ @Override public void outRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+ throw new UnsupportedOperationException();
+ }
+
+ /** {@inheritDoc} */
+ @Nullable @Override public GridNioRecoveryDescriptor outRecoveryDescriptor() {
+ return null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void inRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
throw new UnsupportedOperationException();
}
/** {@inheritDoc} */
- @Nullable @Override public GridNioRecoveryDescriptor recoveryDescriptor() {
+ @Nullable @Override public GridNioRecoveryDescriptor inRecoveryDescriptor() {
return null;
}
/** {@inheritDoc} */
+ @Override public void systemMessage(Object msg) {
+ throw new UnsupportedOperationException();
+ }
+
+ /** {@inheritDoc} */
@Override public String toString() {
return S.toString(GridNioSessionImpl.class, this);
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioWorker.java
new file mode 100644
index 0000000..62985ff
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioWorker.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.util.nio;
+
+import java.util.Collection;
+import java.util.List;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+interface GridNioWorker {
+ /**
+ * @param req Change request.
+ */
+ public void offer(GridNioServer.SessionChangeRequest req);
+
+ /**
+ * @param reqs Change requests.
+ */
+ public void offer(Collection<GridNioServer.SessionChangeRequest> reqs);
+
+ /**
+ * @param ses Session.
+ * @return Session state change requests.
+ */
+ @Nullable public List<GridNioServer.SessionChangeRequest> clearSessionRequests(GridNioSession ses);
+
+ /**
+ * @param ses Session to register write interest for.
+ */
+ public void registerWrite(GridSelectorNioSessionImpl ses);
+}
[21/25] ignite git commit: IGNITE-3220 I/O bottleneck on
server/client cluster configuration Communications optimizations: -
possibility to open separate in/out connections - possibility to have
multiple connections between nodes - implemented NIO sessio
Posted by sb...@apache.org.
IGNITE-3220 I/O bottleneck on server/client cluster configuration
Communications optimizations:
- possibility to open separate in/out connections
- possibility to have multiple connections between nodes
- implemented NIO sessions balancing between NIO threads
- reduced amount of work and blocking calls in NIO threads
Other:
- implemented StripedExecutor for cache messages handling
- added 'io test' messages for IO performance testing
Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/10ade284
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/10ade284
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/10ade284
Branch: refs/heads/ignite-4371
Commit: 10ade2849880790a0fc05a9528ca9b0e1e4617ad
Parents: 065ca4a
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 9 12:28:47 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 9 12:28:50 2016 +0300
----------------------------------------------------------------------
.../ignite/examples/ExampleNodeStartup.java | 2 +-
.../examples/datagrid/CachePutGetExample.java | 2 +-
.../scalar/examples/ScalarJvmCloudExample.scala | 2 +-
.../rest/ClientMemcachedProtocolSelfTest.java | 4 +-
.../rest/protocols/tcp/MockNioSession.java | 25 +-
.../apache/ignite/IgniteSystemProperties.java | 3 +
.../cache/store/CacheLoadOnlyStoreAdapter.java | 6 +-
.../configuration/IgniteConfiguration.java | 50 +-
.../internal/GridEventConsumeHandler.java | 2 +-
.../ignite/internal/GridJobContextImpl.java | 4 +-
.../ignite/internal/GridKernalContext.java | 9 +
.../ignite/internal/GridKernalContextImpl.java | 16 +-
.../internal/GridPerformanceSuggestions.java | 2 +-
.../org/apache/ignite/internal/GridTopic.java | 5 +-
.../ignite/internal/IgniteInternalFuture.java | 11 +
.../apache/ignite/internal/IgniteKernal.java | 85 +-
.../org/apache/ignite/internal/IgnitionEx.java | 32 +-
.../GridClientConnectionManagerAdapter.java | 1 +
.../client/router/impl/GridTcpRouterImpl.java | 1 +
.../managers/communication/GridIoManager.java | 207 ++-
.../managers/communication/GridIoMessage.java | 13 +
.../communication/GridIoMessageFactory.java | 12 +-
.../communication/IgniteIoTestMessage.java | 235 +++
.../processors/cache/GridCacheAdapter.java | 26 +-
.../processors/cache/GridCacheMessage.java | 7 +
.../processors/cache/GridCacheUtils.java | 35 +
.../processors/cache/IgniteCacheProxy.java | 8 +
.../distributed/GridDistributedLockRequest.java | 5 +
.../GridDistributedTxFinishResponse.java | 6 +
.../GridDistributedUnlockRequest.java | 5 +
.../distributed/dht/GridDhtCacheAdapter.java | 3 +-
.../distributed/dht/GridDhtLockResponse.java | 9 +-
.../dht/atomic/GridDhtAtomicCache.java | 5 +-
.../GridDhtAtomicSingleUpdateRequest.java | 5 +
.../dht/atomic/GridDhtAtomicUpdateRequest.java | 5 +
.../atomic/GridNearAtomicFullUpdateRequest.java | 5 +
.../GridNearAtomicSingleUpdateRequest.java | 5 +
.../distributed/near/GridNearGetRequest.java | 5 +
.../local/atomic/GridLocalAtomicCache.java | 3 +
.../query/GridCacheDistributedQueryManager.java | 2 +-
.../cache/query/GridCacheQueryRequest.java | 6 +-
.../transactions/IgniteTxLocalAdapter.java | 8 +-
.../datastreamer/DataStreamProcessor.java | 22 +-
.../internal/processors/igfs/IgfsContext.java | 4 +-
.../processors/igfs/IgfsDataManager.java | 6 +-
.../internal/processors/igfs/IgfsImpl.java | 2 +-
.../internal/processors/odbc/OdbcProcessor.java | 1 +
.../platform/compute/PlatformCompute.java | 6 +
.../tcp/GridTcpMemcachedNioListener.java | 15 +-
.../protocols/tcp/GridTcpRestNioListener.java | 2 +-
.../rest/protocols/tcp/GridTcpRestProtocol.java | 1 +
.../service/GridServiceProcessor.java | 6 +-
.../ignite/internal/util/IgniteUtils.java | 62 +-
.../ignite/internal/util/StripedExecutor.java | 667 +++++++++
.../util/future/GridFinishedFuture.java | 24 +
.../internal/util/future/GridFutureAdapter.java | 15 +-
.../util/future/GridFutureChainListener.java | 30 +-
.../internal/util/ipc/IpcToNioAdapter.java | 2 +-
.../nio/GridAbstractCommunicationClient.java | 12 +-
.../util/nio/GridCommunicationClient.java | 9 +-
.../nio/GridConnectionBytesVerifyFilter.java | 15 +-
.../util/nio/GridNioAsyncNotifyFilter.java | 10 +-
.../internal/util/nio/GridNioCodecFilter.java | 17 +-
.../ignite/internal/util/nio/GridNioFilter.java | 16 +-
.../internal/util/nio/GridNioFilterAdapter.java | 10 +-
.../internal/util/nio/GridNioFilterChain.java | 14 +-
.../ignite/internal/util/nio/GridNioFuture.java | 4 +-
.../util/nio/GridNioRecoveryDescriptor.java | 124 +-
.../ignite/internal/util/nio/GridNioServer.java | 1404 +++++++++++++++---
.../internal/util/nio/GridNioSession.java | 25 +-
.../internal/util/nio/GridNioSessionImpl.java | 65 +-
.../ignite/internal/util/nio/GridNioWorker.java | 48 +
.../util/nio/GridSelectorNioSessionImpl.java | 221 ++-
.../util/nio/GridShmemCommunicationClient.java | 7 +-
.../util/nio/GridTcpNioCommunicationClient.java | 55 +-
.../internal/util/nio/SessionWriteRequest.java | 85 ++
.../internal/util/nio/ssl/GridNioSslFilter.java | 10 +-
.../util/nio/ssl/GridNioSslHandler.java | 4 +-
.../util/tostring/GridToStringBuilder.java | 2 +-
.../communication/tcp/TcpCommunicationSpi.java | 1340 ++++++++++++-----
.../tcp/TcpCommunicationSpiMBean.java | 40 +
.../ignite/spi/discovery/tcp/ServerImpl.java | 14 +-
.../ignite/stream/socket/SocketStreamer.java | 1 +
.../ignite/thread/IgniteThreadFactory.java | 8 +-
.../IgniteSlowClientDetectionSelfTest.java | 1 +
...unicationBalanceMultipleConnectionsTest.java | 28 +
.../IgniteCommunicationBalanceTest.java | 339 +++++
.../communication/IgniteIoTestMessagesTest.java | 95 ++
.../IgniteVariousConnectionNumberTest.java | 166 +++
.../cache/CrossCacheTxRandomOperationsTest.java | 30 +-
...idAbstractCacheInterceptorRebalanceTest.java | 4 +-
...CacheOffHeapMultiThreadedUpdateSelfTest.java | 6 +-
...eAtomicMessageRecovery10ConnectionsTest.java | 28 +
...cMessageRecoveryNoPairedConnectionsTest.java | 47 +
...acheConnectionRecovery10ConnectionsTest.java | 35 +
.../distributed/IgniteCacheCreatePutTest.java | 2 +-
.../IgniteCacheMessageRecoveryAbstractTest.java | 24 +-
.../IgniteCacheMessageWriteTimeoutTest.java | 17 +-
.../dht/IgniteCacheMultiTxLockSelfTest.java | 6 +-
...erNoStripedPoolMultiNodeFullApiSelfTest.java | 35 +
...edNoStripedPoolMultiNodeFullApiSelfTest.java | 35 +
.../TxDeadlockDetectionNoHangsTest.java | 2 +-
.../TxOptimisticDeadlockDetectionTest.java | 29 +-
.../GridServiceProcessorProxySelfTest.java | 2 +-
.../util/future/GridFutureAdapterSelfTest.java | 122 +-
.../nio/impl/GridNioFilterChainSelfTest.java | 32 +-
.../loadtests/nio/GridNioBenchmarkClient.java | 4 +-
.../p2p/GridP2PRecursionTaskSelfTest.java | 2 +-
.../spi/GridTcpSpiForwardingSelfTest.java | 18 +-
.../GridTcpCommunicationSpiAbstractTest.java | 28 +-
...mmunicationSpiConcurrentConnectSelfTest.java | 82 +-
.../GridTcpCommunicationSpiConfigSelfTest.java | 5 +-
...cpCommunicationSpiMultithreadedSelfTest.java | 23 +-
...dTcpCommunicationSpiRecoveryAckSelfTest.java | 9 +-
...tionSpiRecoveryFailureDetectionSelfTest.java | 1 +
...ationSpiRecoveryNoPairedConnectionsTest.java | 28 +
...GridTcpCommunicationSpiRecoverySelfTest.java | 67 +-
...CommunicationRecoveryAckClosureSelfTest.java | 9 +-
.../junits/GridTestKernalContext.java | 4 +-
.../IgniteCacheFullApiSelfTestSuite.java | 6 +
.../ignite/testsuites/IgniteCacheTestSuite.java | 17 +-
.../IgniteSpiCommunicationSelfTestSuite.java | 2 +
.../hadoop/jobtracker/HadoopJobTracker.java | 4 +-
.../HadoopExternalCommunication.java | 5 +-
.../communication/HadoopIpcToNioAdapter.java | 2 +-
.../communication/HadoopMarshallerFilter.java | 6 +-
.../ignite/stream/kafka/KafkaStreamer.java | 2 +-
.../ignite/tools/classgen/ClassesGenerator.java | 8 +-
.../ignite/yardstick/IgniteBenchmarkUtils.java | 6 +-
.../yardstick/cache/CacheEntryEventProbe.java | 2 +-
.../yardstick/cache/IgniteIoTestBenchmark.java | 73 +
.../io/IgniteIoTestAbstractBenchmark.java | 61 +
.../io/IgniteIoTestSendAllBenchmark.java | 32 +
.../io/IgniteIoTestSendRandomBenchmark.java | 35 +
134 files changed, 5935 insertions(+), 998 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/examples/src/main/java/org/apache/ignite/examples/ExampleNodeStartup.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ExampleNodeStartup.java b/examples/src/main/java/org/apache/ignite/examples/ExampleNodeStartup.java
index ad12297..dd8a72b 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ExampleNodeStartup.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ExampleNodeStartup.java
@@ -33,4 +33,4 @@ public class ExampleNodeStartup {
public static void main(String[] args) throws IgniteException {
Ignition.start("examples/config/example-ignite.xml");
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/examples/src/main/java/org/apache/ignite/examples/datagrid/CachePutGetExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/CachePutGetExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/CachePutGetExample.java
index 82a76b8..b9bae5b 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/CachePutGetExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/CachePutGetExample.java
@@ -105,4 +105,4 @@ public class CachePutGetExample {
for (Map.Entry<Integer, String> e : vals.entrySet())
System.out.println("Got entry [key=" + e.getKey() + ", val=" + e.getValue() + ']');
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarJvmCloudExample.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarJvmCloudExample.scala b/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarJvmCloudExample.scala
index 1014726..814bb2e 100644
--- a/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarJvmCloudExample.scala
+++ b/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarJvmCloudExample.scala
@@ -50,7 +50,7 @@ object ScalarJvmCloudExample {
val pool = Executors.newFixedThreadPool(NODES.size)
// Concurrently startup all nodes.
- NODES.foreach(name => pool.submit(new Runnable {
+ NODES.foreach(name => pool.execute(new Runnable {
@impl def run() {
// All defaults.
val cfg = new IgniteConfiguration
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/ClientMemcachedProtocolSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/ClientMemcachedProtocolSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/ClientMemcachedProtocolSelfTest.java
index 0f56c73..c03c06e 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/ClientMemcachedProtocolSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/ClientMemcachedProtocolSelfTest.java
@@ -111,6 +111,8 @@ public class ClientMemcachedProtocolSelfTest extends AbstractRestProcessorSelfTe
Map<String, Object> map = client.getBulk("getKey1", "getKey2");
+ info("Map: " + map);
+
Assert.assertEquals(2, map.size());
Assert.assertEquals("getVal1", map.get("getKey1"));
@@ -443,4 +445,4 @@ public class ClientMemcachedProtocolSelfTest extends AbstractRestProcessorSelfTe
return res;
}
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/MockNioSession.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/MockNioSession.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/MockNioSession.java
index c82c73e..9bc4e7f 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/MockNioSession.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/MockNioSession.java
@@ -18,6 +18,7 @@
package org.apache.ignite.internal.processors.rest.protocols.tcp;
import java.net.InetSocketAddress;
+import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter;
import org.apache.ignite.internal.util.nio.GridNioFinishedFuture;
import org.apache.ignite.internal.util.nio.GridNioFuture;
@@ -111,6 +112,11 @@ public class MockNioSession extends GridMetadataAwareAdapter implements GridNioS
}
/** {@inheritDoc} */
+ @Override public void sendNoFuture(Object msg) throws IgniteCheckedException {
+ // No-op.
+ }
+
+ /** {@inheritDoc} */
@Override public GridNioFuture<Object> resumeReads() {
return null;
}
@@ -131,12 +137,27 @@ public class MockNioSession extends GridMetadataAwareAdapter implements GridNioS
}
/** {@inheritDoc} */
- @Override public void recoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+ @Override public void outRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+ // No-op.
+ }
+
+ /** {@inheritDoc} */
+ @Override public void inRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
// No-op.
}
/** {@inheritDoc} */
- @Nullable @Override public GridNioRecoveryDescriptor recoveryDescriptor() {
+ @Nullable @Override public GridNioRecoveryDescriptor outRecoveryDescriptor() {
return null;
}
+
+ /** {@inheritDoc} */
+ @Nullable @Override public GridNioRecoveryDescriptor inRecoveryDescriptor() {
+ return null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void systemMessage(Object msg) {
+ // No-op.
+ }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
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 b83b5ef..4d85c54 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -483,6 +483,9 @@ public final class IgniteSystemProperties {
@Deprecated
public static final String IGNITE_BINARY_DONT_WRAP_TREE_STRUCTURES = "IGNITE_BINARY_DONT_WRAP_TREE_STRUCTURES";
+ /** */
+ public static final String IGNITE_IO_BALANCE_PERIOD = "IGNITE_IO_BALANCE_PERIOD";
+
/**
* When set to {@code true} fields are written by BinaryMarshaller in sorted order. Otherwise
* the natural order is used.
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/cache/store/CacheLoadOnlyStoreAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheLoadOnlyStoreAdapter.java b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheLoadOnlyStoreAdapter.java
index 7494e37..d3f381e 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheLoadOnlyStoreAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheLoadOnlyStoreAdapter.java
@@ -153,14 +153,14 @@ public abstract class CacheLoadOnlyStoreAdapter<K, V, I> implements CacheStore<K
buf.add(iter.next());
if (buf.size() == batchSize) {
- exec.submit(new Worker(c, buf, args));
+ exec.execute(new Worker(c, buf, args));
buf = new ArrayList<>(batchSize);
}
}
if (!buf.isEmpty())
- exec.submit(new Worker(c, buf, args));
+ exec.execute(new Worker(c, buf, args));
}
catch (RejectedExecutionException ignored) {
// Because of custom RejectedExecutionHandler.
@@ -330,4 +330,4 @@ public abstract class CacheLoadOnlyStoreAdapter<K, V, I> implements CacheStore<K
}
}
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index 75145a3..dcd8a80 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -146,7 +146,7 @@ public class IgniteConfiguration {
public static final int AVAILABLE_PROC_CNT = Runtime.getRuntime().availableProcessors();
/** Default core size of public thread pool. */
- public static final int DFLT_PUBLIC_THREAD_CNT = Math.max(8, AVAILABLE_PROC_CNT) * 2;
+ public static final int DFLT_PUBLIC_THREAD_CNT = Math.max(8, AVAILABLE_PROC_CNT);
/** Default keep alive time for public thread pool. */
@Deprecated
@@ -236,6 +236,12 @@ public class IgniteConfiguration {
/** Async Callback pool size. */
private int callbackPoolSize = DFLT_PUBLIC_THREAD_CNT;
+ /**
+ * Use striped pool for internal requests processing when possible
+ * (e.g. cache requests per-partition striping).
+ */
+ private int stripedPoolSize = DFLT_PUBLIC_THREAD_CNT;
+
/** System pool size. */
private int sysPoolSize = DFLT_SYSTEM_CORE_THREAD_CNT;
@@ -553,6 +559,7 @@ public class IgniteConfiguration {
sndRetryDelay = cfg.getNetworkSendRetryDelay();
sslCtxFactory = cfg.getSslContextFactory();
storeSesLsnrs = cfg.getCacheStoreSessionListenerFactories();
+ stripedPoolSize = cfg.getStripedPoolSize();
svcCfgs = cfg.getServiceConfiguration();
sysPoolSize = cfg.getSystemThreadPoolSize();
timeSrvPortBase = cfg.getTimeServerPortBase();
@@ -712,6 +719,47 @@ public class IgniteConfiguration {
}
/**
+ * Returns striped pool size that should be used for cache requests
+ * processing.
+ * <p>
+ * If set to non-positive value then requests get processed in system pool.
+ * <p>
+ * Striped pool is better for typical cache operations.
+ *
+ * @return Positive value if striped pool should be initialized
+ * with configured number of threads (stripes) and used for requests processing
+ * or non-positive value to process requests in system pool.
+ *
+ * @see #getPublicThreadPoolSize()
+ * @see #getSystemThreadPoolSize()
+ */
+ public int getStripedPoolSize() {
+ return stripedPoolSize;
+ }
+
+ /**
+ * Sets striped pool size that should be used for cache requests
+ * processing.
+ * <p>
+ * If set to non-positive value then requests get processed in system pool.
+ * <p>
+ * Striped pool is better for typical cache operations.
+ *
+ * @param stripedPoolSize Positive value if striped pool should be initialized
+ * with passed in number of threads (stripes) and used for requests processing
+ * or non-positive value to process requests in system pool.
+ * @return {@code this} for chaining.
+ *
+ * @see #getPublicThreadPoolSize()
+ * @see #getSystemThreadPoolSize()
+ */
+ public IgniteConfiguration setStripedPoolSize(int stripedPoolSize) {
+ this.stripedPoolSize = stripedPoolSize;
+
+ return this;
+ }
+
+ /**
* Should return a thread pool size to be used in grid.
* This executor service will be in charge of processing {@link ComputeJob GridJobs}
* and user messages sent to node.
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
index 715f8a5..68d34ce 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
@@ -181,7 +181,7 @@ class GridEventConsumeHandler implements GridContinuousHandler {
notificationQueue.add(new T3<>(nodeId, routineId, evt));
if (!notificationInProgress) {
- ctx.getSystemExecutorService().submit(new Runnable() {
+ ctx.getSystemExecutorService().execute(new Runnable() {
@Override public void run() {
if (!ctx.continuous().lockStopping())
return;
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/GridJobContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobContextImpl.java
index 804d228..dbfa0b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobContextImpl.java
@@ -217,7 +217,7 @@ public class GridJobContextImpl implements ComputeJobContext, Externalizable {
assert execSvc != null;
- execSvc.submit(new Runnable() {
+ execSvc.execute(new Runnable() {
@Override public void run() {
callcc0();
}
@@ -300,4 +300,4 @@ public class GridJobContextImpl implements ComputeJobContext, Externalizable {
@Override public String toString() {
return S.toString(GridJobContextImpl.class, this);
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index ae29223..927944f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -66,6 +66,7 @@ import org.apache.ignite.internal.processors.session.GridTaskSessionProcessor;
import org.apache.ignite.internal.processors.task.GridTaskProcessor;
import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
import org.apache.ignite.internal.util.IgniteExceptionRegistry;
+import org.apache.ignite.internal.util.StripedExecutor;
import org.apache.ignite.internal.util.tostring.GridToStringExclude;
import org.apache.ignite.plugin.PluginNotFoundException;
import org.apache.ignite.plugin.PluginProvider;
@@ -511,6 +512,14 @@ public interface GridKernalContext extends Iterable<GridComponent> {
public ExecutorService getSystemExecutorService();
/**
+ * Executor service that is in charge of processing internal system messages
+ * in stripes (dedicated threads).
+ *
+ * @return Thread pool implementation to be used in grid for internal system messages.
+ */
+ public StripedExecutor getStripedExecutorService();
+
+ /**
* Executor service that is in charge of processing internal and Visor
* {@link org.apache.ignite.compute.ComputeJob GridJobs}.
*
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 94c6448..a2ad1b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -83,6 +83,7 @@ import org.apache.ignite.internal.processors.session.GridTaskSessionProcessor;
import org.apache.ignite.internal.processors.task.GridTaskProcessor;
import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
import org.apache.ignite.internal.util.IgniteExceptionRegistry;
+import org.apache.ignite.internal.util.StripedExecutor;
import org.apache.ignite.internal.util.spring.IgniteSpringHelper;
import org.apache.ignite.internal.util.tostring.GridToStringExclude;
import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -300,6 +301,10 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
/** */
@GridToStringExclude
+ protected StripedExecutor stripedExecSvc;
+
+ /** */
+ @GridToStringExclude
private ExecutorService p2pExecSvc;
/** */
@@ -381,6 +386,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
* @param marshCachePool Marshaller cache pool.
* @param execSvc Public executor service.
* @param sysExecSvc System executor service.
+ * @param stripedExecSvc Striped executor.
* @param p2pExecSvc P2P executor service.
* @param mgmtExecSvc Management executor service.
* @param igfsExecSvc IGFS executor service.
@@ -400,6 +406,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
ExecutorService marshCachePool,
ExecutorService execSvc,
ExecutorService sysExecSvc,
+ StripedExecutor stripedExecSvc,
ExecutorService p2pExecSvc,
ExecutorService mgmtExecSvc,
ExecutorService igfsExecSvc,
@@ -407,7 +414,8 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
ExecutorService affExecSvc,
@Nullable ExecutorService idxExecSvc,
IgniteStripedThreadPoolExecutor callbackExecSvc,
- List<PluginProvider> plugins) throws IgniteCheckedException {
+ List<PluginProvider> plugins
+ ) throws IgniteCheckedException {
assert grid != null;
assert cfg != null;
assert gw != null;
@@ -419,6 +427,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
this.marshCachePool = marshCachePool;
this.execSvc = execSvc;
this.sysExecSvc = sysExecSvc;
+ this.stripedExecSvc = stripedExecSvc;
this.p2pExecSvc = p2pExecSvc;
this.mgmtExecSvc = mgmtExecSvc;
this.igfsExecSvc = igfsExecSvc;
@@ -948,6 +957,11 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
}
/** {@inheritDoc} */
+ @Override public StripedExecutor getStripedExecutorService() {
+ return stripedExecSvc;
+ }
+
+ /** {@inheritDoc} */
@Override public ExecutorService getManagementExecutorService() {
return mgmtExecSvc;
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/GridPerformanceSuggestions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridPerformanceSuggestions.java b/modules/core/src/main/java/org/apache/ignite/internal/GridPerformanceSuggestions.java
index b040a97..5e8e520 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridPerformanceSuggestions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridPerformanceSuggestions.java
@@ -89,4 +89,4 @@ public class GridPerformanceSuggestions {
@Override public String toString() {
return S.toString(GridPerformanceSuggestions.class, this);
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java b/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
index b5608db..24ddcd2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
@@ -97,7 +97,10 @@ public enum GridTopic {
TOPIC_QUERY,
/** */
- TOPIC_TX;
+ TOPIC_TX,
+
+ /** */
+ TOPIC_IO_TEST;
/** Enum values. */
private static final GridTopic[] VALS = values();
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
index b80a755..789556d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
@@ -17,6 +17,7 @@
package org.apache.ignite.internal;
+import java.util.concurrent.Executor;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import org.apache.ignite.IgniteCheckedException;
@@ -133,6 +134,16 @@ public interface IgniteInternalFuture<R> {
public <T> IgniteInternalFuture<T> chain(IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb);
/**
+ * Make a chained future to convert result of this future (when complete) into a new format.
+ * It is guaranteed that done callback will be called only ONCE.
+ *
+ * @param doneCb Done callback that is applied to this future when it finishes to produce chained future result.
+ * @param exec Executor to run callback.
+ * @return Chained future that finishes after this future completes and done callback is called.
+ */
+ public <T> IgniteInternalFuture<T> chain(IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb, Executor exec);
+
+ /**
* @return Error value if future has already been completed with error.
*/
public Throwable error();
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 8fda72f..4972d1f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -61,10 +61,10 @@ import org.apache.ignite.IgniteDataStreamer;
import org.apache.ignite.IgniteEvents;
import org.apache.ignite.IgniteException;
import org.apache.ignite.IgniteFileSystem;
+import org.apache.ignite.IgniteLock;
import org.apache.ignite.IgniteLogger;
import org.apache.ignite.IgniteMessaging;
import org.apache.ignite.IgniteQueue;
-import org.apache.ignite.IgniteLock;
import org.apache.ignite.IgniteScheduler;
import org.apache.ignite.IgniteSemaphore;
import org.apache.ignite.IgniteServices;
@@ -115,7 +115,6 @@ import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor;
import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor;
import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor;
import org.apache.ignite.internal.processors.hadoop.Hadoop;
-import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader;
import org.apache.ignite.internal.processors.hadoop.HadoopProcessorAdapter;
import org.apache.ignite.internal.processors.job.GridJobProcessor;
import org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor;
@@ -139,6 +138,7 @@ import org.apache.ignite.internal.processors.service.GridServiceProcessor;
import org.apache.ignite.internal.processors.session.GridTaskSessionProcessor;
import org.apache.ignite.internal.processors.task.GridTaskProcessor;
import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
+import org.apache.ignite.internal.util.StripedExecutor;
import org.apache.ignite.internal.util.future.GridCompoundFuture;
import org.apache.ignite.internal.util.future.GridFinishedFuture;
import org.apache.ignite.internal.util.future.GridFutureAdapter;
@@ -175,6 +175,7 @@ import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
import org.apache.ignite.thread.IgniteStripedThreadPoolExecutor;
import org.jetbrains.annotations.Nullable;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2;
import static org.apache.ignite.IgniteSystemProperties.IGNITE_CONFIG_URL;
import static org.apache.ignite.IgniteSystemProperties.IGNITE_DAEMON;
import static org.apache.ignite.IgniteSystemProperties.IGNITE_NO_ASCII;
@@ -183,7 +184,6 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_REST_START_ON_CLIE
import static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK;
import static org.apache.ignite.IgniteSystemProperties.IGNITE_STARVATION_CHECK_INTERVAL;
import static org.apache.ignite.IgniteSystemProperties.IGNITE_SUCCESS_FILE;
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2;
import static org.apache.ignite.IgniteSystemProperties.getBoolean;
import static org.apache.ignite.IgniteSystemProperties.snapshot;
import static org.apache.ignite.internal.GridKernalState.DISCONNECTED;
@@ -200,7 +200,6 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_BUILD_VER;
import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_CLIENT_MODE;
import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_CONSISTENCY_CHECK_SKIPPED;
import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_DAEMON;
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_LATE_AFFINITY_ASSIGNMENT;
import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_DEPLOYMENT_MODE;
import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_GRID_NAME;
import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IPS;
@@ -209,11 +208,12 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_JMX_PORT;
import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_JVM_ARGS;
import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_JVM_PID;
import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_LANG_RUNTIME;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_LATE_AFFINITY_ASSIGNMENT;
import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MACS;
import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER;
import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_COMPACT_FOOTER;
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_DFLT_SUID;
import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_BINARY_STRING_SER_VER_2;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_DFLT_SUID;
import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_NODE_CONSISTENT_ID;
import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PEER_CLASSLOADING;
import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PHY_RAM;
@@ -664,6 +664,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
* @param utilityCachePool Utility cache pool.
* @param execSvc Executor service.
* @param sysExecSvc System executor service.
+ * @param stripedExecSvc Striped executor.
* @param p2pExecSvc P2P executor service.
* @param mgmtExecSvc Management executor service.
* @param igfsExecSvc IGFS executor service.
@@ -674,11 +675,13 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
* @throws IgniteCheckedException Thrown in case of any errors.
*/
@SuppressWarnings({"CatchGenericClass", "unchecked"})
- public void start(final IgniteConfiguration cfg,
+ public void start(
+ final IgniteConfiguration cfg,
ExecutorService utilityCachePool,
ExecutorService marshCachePool,
final ExecutorService execSvc,
final ExecutorService sysExecSvc,
+ final StripedExecutor stripedExecSvc,
ExecutorService p2pExecSvc,
ExecutorService mgmtExecSvc,
ExecutorService igfsExecSvc,
@@ -686,7 +689,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
ExecutorService affExecSvc,
@Nullable ExecutorService idxExecSvc,
IgniteStripedThreadPoolExecutor callbackExecSvc,
- GridAbsClosure errHnd)
+ GridAbsClosure errHnd
+ )
throws IgniteCheckedException
{
gw.compareAndSet(null, new GridKernalGatewayImpl(cfg.getGridName()));
@@ -786,6 +790,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
marshCachePool,
execSvc,
sysExecSvc,
+ stripedExecSvc,
p2pExecSvc,
mgmtExecSvc,
igfsExecSvc,
@@ -793,7 +798,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
affExecSvc,
idxExecSvc,
callbackExecSvc,
- plugins);
+ plugins
+ );
cfg.getMarshaller().setContext(ctx.marshallerContext());
@@ -987,24 +993,51 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
starveTask = ctx.timeout().schedule(new Runnable() {
/** Last completed task count. */
- private long lastCompletedCnt;
+ private long lastCompletedCntPub;
+
+ /** Last completed task count. */
+ private long lastCompletedCntSys;
@Override public void run() {
- if (!(execSvc instanceof ThreadPoolExecutor))
- return;
+ if (execSvc instanceof ThreadPoolExecutor) {
+ ThreadPoolExecutor exec = (ThreadPoolExecutor)execSvc;
+
+ lastCompletedCntPub = checkPoolStarvation(exec, lastCompletedCntPub, "public");
+ }
- ThreadPoolExecutor exec = (ThreadPoolExecutor)execSvc;
+ if (sysExecSvc instanceof ThreadPoolExecutor) {
+ ThreadPoolExecutor exec = (ThreadPoolExecutor)sysExecSvc;
+ lastCompletedCntSys = checkPoolStarvation(exec, lastCompletedCntSys, "system");
+ }
+
+ if (stripedExecSvc != null)
+ stripedExecSvc.checkStarvation();
+ }
+
+ /**
+ * @param exec Thread pool executor to check.
+ * @param lastCompletedCnt Last completed tasks count.
+ * @param pool Pool name for message.
+ * @return Current completed tasks count.
+ */
+ private long checkPoolStarvation(
+ ThreadPoolExecutor exec,
+ long lastCompletedCnt,
+ String pool
+ ) {
long completedCnt = exec.getCompletedTaskCount();
// If all threads are active and no task has completed since last time and there is
// at least one waiting request, then it is possible starvation.
if (exec.getPoolSize() == exec.getActiveCount() && completedCnt == lastCompletedCnt &&
!exec.getQueue().isEmpty())
- LT.warn(log, "Possible thread pool starvation detected (no task completed in last " +
- interval + "ms, is executorService pool size large enough?)");
+ LT.warn(
+ log,
+ "Possible thread pool starvation detected (no task completed in last " +
+ interval + "ms, is " + pool + " thread pool size large enough?)");
- lastCompletedCnt = completedCnt;
+ return completedCnt;
}
}, interval, interval);
}
@@ -1129,6 +1162,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
}, longOpDumpTimeout, longOpDumpTimeout);
}
+ ctx.performance().add("Disable assertions (remove '-ea' from JVM options)", !U.assertionsEnabled());
+
ctx.performance().logSuggestions(log, gridName);
U.quietAndInfo(log, "To start Console Management & Monitoring run ignitevisorcmd.{sh|bat}");
@@ -3519,6 +3554,26 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
}
}
+ /**
+ * @param node Node.
+ * @param payload Message payload.
+ * @param procFromNioThread If {@code true} message is processed from NIO thread.
+ * @return Response future.
+ */
+ public IgniteInternalFuture sendIoTest(ClusterNode node, byte[] payload, boolean procFromNioThread) {
+ return ctx.io().sendIoTest(node, payload, procFromNioThread);
+ }
+
+ /**
+ * @param nodes Nodes.
+ * @param payload Message payload.
+ * @param procFromNioThread If {@code true} message is processed from NIO thread.
+ * @return Response future.
+ */
+ public IgniteInternalFuture sendIoTest(List<ClusterNode> nodes, byte[] payload, boolean procFromNioThread) {
+ return ctx.io().sendIoTest(nodes, payload, procFromNioThread);
+ }
+
/** {@inheritDoc} */
@Override public String toString() {
return S.toString(IgniteKernal.class, this);
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index b3a9eec..f32a753 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -66,6 +66,7 @@ import org.apache.ignite.internal.processors.igfs.IgfsUtils;
import org.apache.ignite.internal.processors.resource.GridSpringResourceContext;
import org.apache.ignite.internal.util.GridConcurrentHashSet;
import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.StripedExecutor;
import org.apache.ignite.internal.util.spring.IgniteSpringHelper;
import org.apache.ignite.internal.util.typedef.CA;
import org.apache.ignite.internal.util.typedef.F;
@@ -1459,6 +1460,9 @@ public class IgnitionEx {
/** System executor service. */
private ThreadPoolExecutor sysExecSvc;
+ /** */
+ private StripedExecutor stripedExecSvc;
+
/** Management executor service. */
private ThreadPoolExecutor mgmtExecSvc;
@@ -1652,8 +1656,6 @@ public class IgnitionEx {
execSvc.allowCoreThreadTimeOut(true);
- // Note that since we use 'LinkedBlockingQueue', number of
- // maximum threads has no effect.
validateThreadPoolSize(cfg.getSystemThreadPoolSize(), "system");
sysExecSvc = new IgniteThreadPoolExecutor(
@@ -1666,6 +1668,9 @@ public class IgnitionEx {
sysExecSvc.allowCoreThreadTimeOut(true);
+ if (cfg.getStripedPoolSize() > 0)
+ stripedExecSvc = new StripedExecutor(cfg.getStripedPoolSize(), cfg.getGridName(), "sys", log);
+
// Note that since we use 'LinkedBlockingQueue', number of
// maximum threads has no effect.
// Note, that we do not pre-start threads here as management pool may
@@ -1791,13 +1796,26 @@ public class IgnitionEx {
// Init here to make grid available to lifecycle listeners.
grid = grid0;
- grid0.start(myCfg, utilityCacheExecSvc, marshCacheExecSvc, execSvc, sysExecSvc, p2pExecSvc, mgmtExecSvc,
- igfsExecSvc, restExecSvc, affExecSvc, idxExecSvc, callbackExecSvc,
+ grid0.start(
+ myCfg,
+ utilityCacheExecSvc,
+ marshCacheExecSvc,
+ execSvc,
+ sysExecSvc,
+ stripedExecSvc,
+ p2pExecSvc,
+ mgmtExecSvc,
+ igfsExecSvc,
+ restExecSvc,
+ affExecSvc,
+ idxExecSvc,
+ callbackExecSvc,
new CA() {
@Override public void apply() {
startLatch.countDown();
}
- });
+ }
+ );
state = STARTED;
@@ -2415,6 +2433,10 @@ public class IgnitionEx {
sysExecSvc = null;
+ U.shutdownNow(getClass(), stripedExecSvc, log);
+
+ stripedExecSvc = null;
+
U.shutdownNow(getClass(), mgmtExecSvc, log);
mgmtExecSvc = null;
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
index 6ea7c22..12baee0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
@@ -200,6 +200,7 @@ public abstract class GridClientConnectionManagerAdapter implements GridClientCo
.socketSendBufferSize(0)
.idleTimeout(Long.MAX_VALUE)
.gridName(routerClient ? "routerClient" : "gridClient")
+ .serverName("tcp-client")
.daemon(cfg.isDaemon())
.build();
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java
index 06a4929..3566830 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java
@@ -258,6 +258,7 @@ public class GridTcpRouterImpl implements GridTcpRouter, GridTcpRouterMBean, Lif
.logger(log)
.selectorCount(Runtime.getRuntime().availableProcessors())
.gridName(gridName)
+ .serverName("router")
.tcpNoDelay(tcpNoDelay)
.directBuffer(false)
.byteOrder(ByteOrder.nativeOrder())
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index 3df29cf..7ef7bc0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -26,15 +26,17 @@ import java.util.Map;
import java.util.Map.Entry;
import java.util.Queue;
import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
-
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteException;
import org.apache.ignite.cluster.ClusterNode;
@@ -44,6 +46,7 @@ import org.apache.ignite.internal.GridKernalContext;
import org.apache.ignite.internal.GridTopic;
import org.apache.ignite.internal.IgniteComponentType;
import org.apache.ignite.internal.IgniteDeploymentCheckedException;
+import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.direct.DirectMessageReader;
import org.apache.ignite.internal.direct.DirectMessageWriter;
import org.apache.ignite.internal.managers.GridManagerAdapter;
@@ -55,6 +58,8 @@ import org.apache.ignite.internal.processors.pool.PoolProcessor;
import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashSet;
import org.apache.ignite.internal.util.GridSpinReadWriteLock;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
import org.apache.ignite.internal.util.lang.GridTuple3;
import org.apache.ignite.internal.util.tostring.GridToStringInclude;
import org.apache.ignite.internal.util.typedef.F;
@@ -83,6 +88,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
import static org.apache.ignite.internal.GridTopic.TOPIC_COMM_USER;
+import static org.apache.ignite.internal.GridTopic.TOPIC_IO_TEST;
import static org.apache.ignite.internal.managers.communication.GridIoPolicy.AFFINITY_POOL;
import static org.apache.ignite.internal.managers.communication.GridIoPolicy.IDX_POOL;
import static org.apache.ignite.internal.managers.communication.GridIoPolicy.IGFS_POOL;
@@ -176,6 +182,12 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
/** Stopping flag. */
private boolean stopping;
+ /** */
+ private final AtomicReference<ConcurrentHashMap<Long, IoTestFuture>> ioTestMap = new AtomicReference<>();
+
+ /** */
+ private final AtomicLong ioTestId = new AtomicLong();
+
/**
* @param ctx Grid kernal context.
*/
@@ -297,6 +309,114 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
if (log.isDebugEnabled())
log.debug(startInfo());
+
+ addMessageListener(GridTopic.TOPIC_IO_TEST, new GridMessageListener() {
+ @Override public void onMessage(UUID nodeId, Object msg) {
+ ClusterNode node = ctx.discovery().node(nodeId);
+
+ if (node == null)
+ return;
+
+ IgniteIoTestMessage msg0 = (IgniteIoTestMessage)msg;
+
+ if (msg0.request()) {
+ IgniteIoTestMessage res = new IgniteIoTestMessage(msg0.id(), false, null);
+
+ res.flags(msg0.flags());
+
+ try {
+ send(node, GridTopic.TOPIC_IO_TEST, res, GridIoPolicy.SYSTEM_POOL);
+ }
+ catch (IgniteCheckedException e) {
+ U.error(log, "Failed to send IO test response [msg=" + msg0 + "]", e);
+ }
+ }
+ else {
+ IoTestFuture fut = ioTestMap().get(msg0.id());
+
+ if (fut == null)
+ U.warn(log, "Failed to find IO test future [msg=" + msg0 + ']');
+ else
+ fut.onResponse();
+ }
+ }
+ });
+ }
+
+ /**
+ * @param nodes Nodes.
+ * @param payload Payload.
+ * @param procFromNioThread If {@code true} message is processed from NIO thread.
+ * @return Response future.
+ */
+ public IgniteInternalFuture sendIoTest(List<ClusterNode> nodes, byte[] payload, boolean procFromNioThread) {
+ long id = ioTestId.getAndIncrement();
+
+ IoTestFuture fut = new IoTestFuture(id, nodes.size());
+
+ IgniteIoTestMessage msg = new IgniteIoTestMessage(id, true, payload);
+
+ msg.processFromNioThread(procFromNioThread);
+
+ ioTestMap().put(id, fut);
+
+ for (int i = 0; i < nodes.size(); i++) {
+ ClusterNode node = nodes.get(i);
+
+ try {
+ send(node, GridTopic.TOPIC_IO_TEST, msg, GridIoPolicy.SYSTEM_POOL);
+ }
+ catch (IgniteCheckedException e) {
+ ioTestMap().remove(msg.id());
+
+ return new GridFinishedFuture(e);
+ }
+ }
+
+ return fut;
+ }
+
+ /**
+ * @param node Node.
+ * @param payload Payload.
+ * @param procFromNioThread If {@code true} message is processed from NIO thread.
+ * @return Response future.
+ */
+ public IgniteInternalFuture sendIoTest(ClusterNode node, byte[] payload, boolean procFromNioThread) {
+ long id = ioTestId.getAndIncrement();
+
+ IoTestFuture fut = new IoTestFuture(id, 1);
+
+ IgniteIoTestMessage msg = new IgniteIoTestMessage(id, true, payload);
+
+ msg.processFromNioThread(procFromNioThread);
+
+ ioTestMap().put(id, fut);
+
+ try {
+ send(node, GridTopic.TOPIC_IO_TEST, msg, GridIoPolicy.SYSTEM_POOL);
+ }
+ catch (IgniteCheckedException e) {
+ ioTestMap().remove(msg.id());
+
+ return new GridFinishedFuture(e);
+ }
+
+ return fut;
+ }
+
+ /**
+ * @return IO test futures map.
+ */
+ private ConcurrentHashMap<Long, IoTestFuture> ioTestMap() {
+ ConcurrentHashMap<Long, IoTestFuture> map = ioTestMap.get();
+
+ if (map == null) {
+ if (!ioTestMap.compareAndSet(null, map = new ConcurrentHashMap<>()))
+ map = ioTestMap.get();
+ }
+
+ return map;
}
/** {@inheritDoc} */
@@ -514,16 +634,6 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
return;
}
- // Check discovery.
- ClusterNode node = ctx.discovery().node(nodeId);
-
- if (node == null) {
- if (log.isDebugEnabled())
- log.debug("Ignoring message from dead node [senderId=" + nodeId + ", msg=" + msg + ']');
-
- return; // We can't receive messages from non-discovered ones.
- }
-
if (msg.topic() == null) {
int topicOrd = msg.topicOrdinal();
@@ -678,8 +788,31 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
msgC.run();
}
}
+
+ @Override public String toString() {
+ return "Message closure [msg=" + msg + ']';
+ }
};
+ if (msg.topicOrdinal() == TOPIC_IO_TEST.ordinal()) {
+ IgniteIoTestMessage msg0 = (IgniteIoTestMessage)msg.message();
+
+ if (msg0.processFromNioThread()) {
+ c.run();
+
+ return;
+ }
+ }
+
+ if (ctx.config().getStripedPoolSize() > 0 &&
+ plc == GridIoPolicy.SYSTEM_POOL &&
+ msg.partition() != Integer.MIN_VALUE
+ ) {
+ ctx.getStripedExecutorService().execute(msg.partition(), c);
+
+ return;
+ }
+
try {
pools.poolForPolicy(plc).execute(c);
}
@@ -2460,4 +2593,56 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
return S.toString(DelayedMessage.class, this, super.toString());
}
}
+
+ /**
+ *
+ */
+ private class IoTestFuture extends GridFutureAdapter<Object> {
+ /** */
+ private final long id;
+
+ /** */
+ private int cntr;
+
+ /**
+ * @param id ID.
+ * @param cntr Counter.
+ */
+ IoTestFuture(long id, int cntr) {
+ assert cntr > 0 : cntr;
+
+ this.id = id;
+ this.cntr = cntr;
+ }
+
+ /**
+ *
+ */
+ void onResponse() {
+ boolean complete;
+
+ synchronized (this) {
+ complete = --cntr == 0;
+ }
+
+ if (complete)
+ onDone();
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean onDone(@Nullable Object res, @Nullable Throwable err) {
+ if (super.onDone(res, err)) {
+ ioTestMap().remove(id);
+
+ return true;
+ }
+
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(IoTestFuture.class, this);
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
index b28ced2..b1a26e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.managers.communication;
import java.io.Externalizable;
import java.nio.ByteBuffer;
import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.processors.cache.GridCacheMessage;
import org.apache.ignite.internal.util.tostring.GridToStringInclude;
import org.apache.ignite.internal.util.typedef.internal.S;
import org.apache.ignite.plugin.extensions.communication.Message;
@@ -321,6 +322,18 @@ public class GridIoMessage implements Message {
return 7;
}
+ /**
+ * Get single partition for this message (if applicable).
+ *
+ * @return Partition ID.
+ */
+ public int partition() {
+ if (msg instanceof GridCacheMessage)
+ return ((GridCacheMessage)msg).partition();
+ else
+ return Integer.MIN_VALUE;
+ }
+
/** {@inheritDoc} */
@Override public String toString() {
return S.toString(GridIoMessage.class, this);
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index 504e683..b1fe910 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@ -171,6 +171,16 @@ public class GridIoMessageFactory implements MessageFactory {
Message msg = null;
switch (type) {
+ case -44:
+ msg = new TcpCommunicationSpi.HandshakeMessage2();
+
+ break;
+
+ case -43:
+ msg = new IgniteIoTestMessage();
+
+ break;
+
case -42:
msg = new HadoopDirectShuffleMessage();
@@ -816,7 +826,7 @@ public class GridIoMessageFactory implements MessageFactory {
break;
- // [-3..119] [124..127] [-36]- this
+ // [-3..119] [124..127] [-36..-44]- this
// [120..123] - DR
// [-4..-22, -30..-35] - SQL
default:
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
new file mode 100644
index 0000000..77aaa09
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.managers.communication;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+import java.nio.ByteBuffer;
+
+/**
+ *
+ */
+public class IgniteIoTestMessage implements Message {
+ /** */
+ private static byte FLAG_PROC_FROM_NIO = 1;
+
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ private long id;
+
+ /** */
+ private byte flags;
+
+ /** */
+ private boolean req;
+
+ /** */
+ private byte payload[];
+
+ /**
+ *
+ */
+ IgniteIoTestMessage() {
+ // No-op.
+ }
+
+ /**
+ * @param id Message ID.
+ * @param req Request flag.
+ * @param payload Payload.
+ */
+ IgniteIoTestMessage(long id, boolean req, byte[] payload) {
+ this.id = id;
+ this.req = req;
+ this.payload = payload;
+ }
+
+ /**
+ * @return {@code True} if message should be processed from NIO thread
+ * (otherwise message is submitted to system pool).
+ */
+ boolean processFromNioThread() {
+ return isFlag(FLAG_PROC_FROM_NIO);
+ }
+
+ /**
+ * @param procFromNioThread {@code True} if message should be processed from NIO thread.
+ */
+ void processFromNioThread(boolean procFromNioThread) {
+ setFlag(procFromNioThread, FLAG_PROC_FROM_NIO);
+ }
+
+ /**
+ * @param flags Flags.
+ */
+ public void flags(byte flags) {
+ this.flags = flags;
+ }
+
+ /**
+ * @return Flags.
+ */
+ public byte flags() {
+ return flags;
+ }
+
+ /**
+ * Sets flag mask.
+ *
+ * @param flag Set or clear.
+ * @param mask Mask.
+ */
+ private void setFlag(boolean flag, int mask) {
+ flags = flag ? (byte)(flags | mask) : (byte)(flags & ~mask);
+ }
+
+ /**
+ * Reads flag mask.
+ *
+ * @param mask Mask to read.
+ * @return Flag value.
+ */
+ private boolean isFlag(int mask) {
+ return (flags & mask) != 0;
+ }
+
+ /**
+ * @return {@code true} if this is request.
+ */
+ public boolean request() {
+ return req;
+ }
+
+ /**
+ * @return ID.
+ */
+ public long id() {
+ return id;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+ writer.setBuffer(buf);
+
+ if (!writer.isHeaderWritten()) {
+ if (!writer.writeHeader(directType(), fieldsCount()))
+ return false;
+
+ writer.onHeaderWritten();
+ }
+
+ switch (writer.state()) {
+ case 0:
+ if (!writer.writeByte("flags", flags))
+ return false;
+
+ writer.incrementState();
+
+ case 1:
+ if (!writer.writeLong("id", id))
+ return false;
+
+ writer.incrementState();
+
+ case 2:
+ if (!writer.writeByteArray("payload", payload))
+ return false;
+
+ writer.incrementState();
+
+ case 3:
+ if (!writer.writeBoolean("req", req))
+ return false;
+
+ writer.incrementState();
+
+ }
+
+ return true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+ reader.setBuffer(buf);
+
+ if (!reader.beforeMessageRead())
+ return false;
+
+ switch (reader.state()) {
+ case 0:
+ flags = reader.readByte("flags");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 1:
+ id = reader.readLong("id");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 2:
+ payload = reader.readByteArray("payload");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 3:
+ req = reader.readBoolean("req");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ }
+
+ return reader.afterMessageRead(IgniteIoTestMessage.class);
+ }
+
+ /** {@inheritDoc} */
+ @Override public byte directType() {
+ return -43;
+ }
+
+ /** {@inheritDoc} */
+ @Override public byte fieldsCount() {
+ return 4;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onAckReceived() {
+ // No-op.
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(IgniteIoTestMessage.class, this);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 0f39c69..1db3931 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -288,6 +288,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
/** Asynchronous operations limit semaphore. */
private Semaphore asyncOpsSem;
+ /** */
+ protected volatile boolean asyncToggled;
+
/** {@inheritDoc} */
@Override public String name() {
return cacheCfg.getName();
@@ -364,6 +367,18 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
}
/**
+ * Toggles async flag if someone calls {@code withAsync()}
+ * on proxy and since that we have to properly handle all cache
+ * operations (sync and async) to put them in proper sequence.
+ *
+ * TODO: https://issues.apache.org/jira/browse/IGNITE-4393
+ */
+ void toggleAsync() {
+ if (!asyncToggled)
+ asyncToggled = true;
+ }
+
+ /**
* Prints memory stats.
*/
public void printMemoryStats() {
@@ -1134,7 +1149,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
execSvc = Executors.newFixedThreadPool(jobs.size() - 1);
for (int i = 1; i < jobs.size(); i++)
- execSvc.submit(jobs.get(i));
+ execSvc.execute(jobs.get(i));
}
try {
@@ -2534,6 +2549,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
* @return Put future.
*/
public IgniteInternalFuture<Boolean> putAsync(K key, V val, @Nullable CacheEntryPredicate filter) {
+ A.notNull(key, "key", val, "val");
+
final boolean statsEnabled = ctx.config().isStatisticsEnabled();
final long start = statsEnabled ? System.nanoTime() : 0L;
@@ -2554,8 +2571,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
*/
public IgniteInternalFuture<Boolean> putAsync0(final K key, final V val,
@Nullable final CacheEntryPredicate filter) {
- A.notNull(key, "key", val, "val");
-
if (keyCheck)
validateCacheKey(key);
@@ -4592,6 +4607,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
* @return Failed future if waiting was interrupted.
*/
@Nullable protected <T> IgniteInternalFuture<T> asyncOpAcquire() {
+ if (!asyncToggled)
+ return null;
+
try {
if (asyncOpsSem != null)
asyncOpsSem.acquire();
@@ -4610,7 +4628,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
* Releases asynchronous operations permit, if limited.
*/
protected void asyncOpRelease() {
- if (asyncOpsSem != null)
+ if (asyncOpsSem != null && asyncToggled)
asyncOpsSem.release();
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
index 71f99d3..0646d5a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
@@ -129,6 +129,13 @@ public abstract class GridCacheMessage implements Message {
}
/**
+ * @return Partition ID this message is targeted to or {@code -1} if it cannot be determined.
+ */
+ public int partition() {
+ return -1;
+ }
+
+ /**
* If class loading error occurred during unmarshalling and {@link #ignoreClassErrors()} is
* set to {@code true}, then the error will be passed into this method.
*
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index d32f4c1..47abf2f 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
@@ -110,6 +110,41 @@ import static org.apache.ignite.internal.processors.cache.GridCacheOperation.REA
* Cache utility methods.
*/
public class GridCacheUtils {
+ /** Cheat cache ID for debugging and benchmarking purposes. */
+ public static final int cheatCacheId;
+
+ /*
+ *
+ */
+ static {
+ String cheatCache = System.getProperty("CHEAT_CACHE");
+
+ if (cheatCache != null) {
+ cheatCacheId = cheatCache.hashCode();
+
+ if (cheatCacheId == 0)
+ throw new RuntimeException();
+
+ System.out.println(">>> Cheat cache ID [id=" + cheatCacheId + ", name=" + cheatCache + ']');
+ }
+ else
+ cheatCacheId = 0;
+ }
+
+ /**
+ * Quickly checks if passed in cache ID is a "cheat cache ID" set by -DCHEAT_CACHE=user_cache_name
+ * and resolved in static block above.
+ *
+ * FOR DEBUGGING AND TESTING PURPOSES!
+ *
+ * @param id Cache ID to check.
+ * @return {@code True} if this is cheat cache ID.
+ */
+ @Deprecated
+ public static boolean cheatCache(int id) {
+ return cheatCacheId != 0 && id == cheatCacheId;
+ }
+
/** Hadoop syste cache name. */
public static final String SYS_CACHE_HADOOP_MR = "ignite-hadoop-mr-sys-cache";
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index f03a3b5..90898f9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -334,6 +334,14 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
}
/** {@inheritDoc} */
+ @Override public IgniteCache<K, V> withAsync() {
+ if (delegate instanceof GridCacheAdapter)
+ ((GridCacheAdapter)delegate).toggleAsync();
+
+ return super.withAsync();
+ }
+
+ /** {@inheritDoc} */
@Override public IgniteCache<K, V> withSkipStore() {
return skipStore();
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
index 9639a9a..a671296 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
@@ -310,6 +310,11 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
return keys;
}
+ /** {@inheritDoc} */
+ @Override public int partition() {
+ return partIds != null && !partIds.isEmpty() ? partIds.get(0) : -1;
+ }
+
/**
* @return Max lock wait time.
*/
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
index 109d665..c5cf332 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
@@ -86,6 +86,12 @@ public class GridDistributedTxFinishResponse extends GridCacheMessage {
}
/** {@inheritDoc} */
+ @Override public int partition() {
+ // TODO https://issues.apache.org/jira/browse/IGNITE-4371
+ return Integer.MIN_VALUE;
+ }
+
+ /** {@inheritDoc} */
@Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
writer.setBuffer(buf);
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java
index df6acdd..5d70ec1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java
@@ -89,6 +89,11 @@ public class GridDistributedUnlockRequest extends GridDistributedBaseMessage {
partIds.add(key.partition());
}
+ /** {@inheritDoc} */
+ @Override public int partition() {
+ return partIds != null && !partIds.isEmpty() ? partIds.get(0) : -1;
+ }
+
/** {@inheritDoc}
* @param ctx*/
@Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException {
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index 35e6267..519d0fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -218,7 +218,8 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
@Override public void onKernalStart() throws IgniteCheckedException {
super.onKernalStart();
- preldr.onKernalStart();
+ if (preldr != null)
+ preldr.onKernalStart();
}
/** {@inheritDoc} */
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java
index 1e92b54..63e3309 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java
@@ -22,6 +22,7 @@ import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
+import java.util.HashSet;
import java.util.List;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.internal.GridDirectCollection;
@@ -31,7 +32,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockResponse;
import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.util.GridLeanSet;
import org.apache.ignite.internal.util.tostring.GridToStringInclude;
import org.apache.ignite.internal.util.typedef.internal.S;
import org.apache.ignite.lang.IgniteUuid;
@@ -57,7 +57,7 @@ public class GridDhtLockResponse extends GridDistributedLockResponse {
/** Invalid partitions. */
@GridToStringInclude
@GridDirectCollection(int.class)
- private Collection<Integer> invalidParts = new GridLeanSet<>();
+ private Collection<Integer> invalidParts;
/** Preload entries. */
@GridDirectCollection(GridCacheEntryInfo.class)
@@ -127,6 +127,9 @@ public class GridDhtLockResponse extends GridDistributedLockResponse {
* @param part Invalid partition.
*/
public void addInvalidPartition(int part) {
+ if (invalidParts == null)
+ invalidParts = new HashSet<>();
+
invalidParts.add(part);
}
@@ -134,7 +137,7 @@ public class GridDhtLockResponse extends GridDistributedLockResponse {
* @return Invalid partitions.
*/
public Collection<Integer> invalidPartitions() {
- return invalidParts;
+ return invalidParts == null ? Collections.<Integer>emptySet() : invalidParts;
}
/**
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 940c74e..0e60ff4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -613,8 +613,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
/** {@inheritDoc} */
@SuppressWarnings("unchecked")
@Override public IgniteInternalFuture<Boolean> putAsync0(K key, V val, @Nullable CacheEntryPredicate filter) {
- A.notNull(key, "key", val, "val");
-
return updateAsync0(
key,
val,
@@ -814,6 +812,9 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
*/
@SuppressWarnings("unchecked")
protected <T> IgniteInternalFuture<T> asyncOp(final CO<IgniteInternalFuture<T>> op) {
+ if (!asyncToggled)
+ return op.apply();
+
IgniteInternalFuture<T> fail = asyncOpAcquire();
if (fail != null)
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java
index a03d948..0af7cf5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java
@@ -235,6 +235,11 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
}
/** {@inheritDoc} */
+ @Override public int partition() {
+ return partId;
+ }
+
+ /** {@inheritDoc} */
@Override public int partitionId(int idx) {
assert idx == 0 : idx;
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
index f2fbb0e..1854e52 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
@@ -487,6 +487,11 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
}
/** {@inheritDoc} */
+ @Override public int partition() {
+ return partIds != null && !partIds.isEmpty() ? partIds.get(0) : -1;
+ }
+
+ /** {@inheritDoc} */
@Override public long conflictExpireTime(int idx) {
if (conflictExpireTimes != null) {
assert idx >= 0 && idx < conflictExpireTimes.size();
[20/25] ignite git commit: IGNITE-3220 I/O bottleneck on
server/client cluster configuration Communications optimizations: -
possibility to open separate in/out connections - possibility to have
multiple connections between nodes - implemented NIO sessio
Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java
index 1b11688..87d9225 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java
@@ -611,6 +611,11 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
}
/** {@inheritDoc} */
+ @Override public int partition() {
+ return partIds != null && !partIds.isEmpty() ? partIds.get(0) : -1;
+ }
+
+ /** {@inheritDoc} */
@Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
writer.setBuffer(buf);
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
index 1c1addd..c3e9fbe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
@@ -125,6 +125,11 @@ public class GridNearAtomicSingleUpdateRequest extends GridNearAtomicAbstractSin
);
}
+ /** {@inheritDoc} */
+ @Override public int partition() {
+ return partId;
+ }
+
/**
* @param key Key to add.
* @param val Optional update value.
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java
index fa7f367..4272a4d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java
@@ -244,6 +244,11 @@ public class GridNearGetRequest extends GridCacheMessage implements GridCacheDep
return accessTtl;
}
+ /** {@inheritDoc} */
+ @Override public int partition() {
+ return partIds != null && !partIds.isEmpty() ? partIds.get(0) : -1;
+ }
+
/**
* @param ctx Cache context.
* @throws IgniteCheckedException If failed.
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
index a419887..bc16ff4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -1585,6 +1585,9 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
*/
@SuppressWarnings("unchecked")
protected IgniteInternalFuture asyncOp(final Callable<?> op) {
+ if (!asyncToggled)
+ return ctx.closures().callLocalSafe(op);
+
IgniteInternalFuture fail = asyncOpAcquire();
if (fail != null)
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
index d34047e..eb5e214 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
@@ -272,7 +272,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
false,
null,
req.keyValueFilter(),
- req.partition(),
+ req.partition() == -1 ? null : req.partition(),
req.className(),
req.clause(),
req.includeMetaData(),
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
index 60c4662..9f965d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
@@ -121,7 +121,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
private int taskHash;
/** Partition. */
- private int part;
+ private int part = -1;
/** */
private AffinityTopologyVersion topVer;
@@ -478,8 +478,8 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
/**
* @return partition.
*/
- @Nullable public Integer partition() {
- return part == -1 ? null : part;
+ public int partition() {
+ return part;
}
/** {@inheritDoc} */
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 6d21dcf..393fb1a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -391,7 +391,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
/** {@inheritDoc} */
@Override public IgniteInternalFuture<Void> loadMissing(
final GridCacheContext cacheCtx,
- AffinityTopologyVersion topVer,
+ final AffinityTopologyVersion topVer,
final boolean readThrough,
boolean async,
final Collection<KeyCacheObject> keys,
@@ -472,7 +472,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
CacheObject cacheVal = cacheCtx.toCacheObject(val);
while (true) {
- GridCacheEntryEx entry = cacheCtx.cache().entryEx(key);
+ GridCacheEntryEx entry = cacheCtx.cache().entryEx(key, topVer);
try {
GridCacheVersion setVer = entry.versionedValue(cacheVal, ver, null);
@@ -1507,7 +1507,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
assert txEntry != null || readCommitted() || skipVals;
- GridCacheEntryEx e = txEntry == null ? entryEx(cacheCtx, txKey) : txEntry.cached();
+ GridCacheEntryEx e = txEntry == null ? entryEx(cacheCtx, txKey, topVer) : txEntry.cached();
if (readCommitted() || skipVals) {
cacheCtx.evicts().touch(e, topologyVersion());
@@ -1658,7 +1658,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
IgniteTxLocalAdapter.this,
/*swap*/cacheCtx.isSwapOrOffheapEnabled(),
/*unmarshal*/true,
- /**update-metrics*/true,
+ /*update-metrics*/true,
/*event*/!skipVals,
CU.subjectId(IgniteTxLocalAdapter.this, cctx),
transformClo,
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
index 32fda87..fee4dd6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
@@ -328,6 +328,8 @@ public class DataStreamProcessor<K, V> extends GridProcessorAdapter {
if (!allowOverwrite)
cctx.topology().readLock();
+ GridDhtTopologyFuture topWaitFut = null;
+
try {
GridDhtTopologyFuture fut = cctx.topologyVersionFuture();
@@ -352,19 +354,25 @@ public class DataStreamProcessor<K, V> extends GridProcessorAdapter {
waitFut = allowOverwrite ? null : cctx.mvcc().addDataStreamerFuture(topVer);
}
- else {
- fut.listen(new IgniteInClosure<IgniteInternalFuture<AffinityTopologyVersion>>() {
- @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> e) {
- localUpdate(nodeId, req, updater, topic);
- }
- });
- }
+ else
+ topWaitFut = fut;
}
finally {
if (!allowOverwrite)
cctx.topology().readUnlock();
}
+ if (topWaitFut != null) {
+ // Need call 'listen' after topology read lock is released.
+ topWaitFut.listen(new IgniteInClosure<IgniteInternalFuture<AffinityTopologyVersion>>() {
+ @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> e) {
+ localUpdate(nodeId, req, updater, topic);
+ }
+ });
+
+ return;
+ }
+
if (job != null) {
try {
job.call();
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
index 3405b53..4c037b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
@@ -226,7 +226,7 @@ public class IgfsContext {
*/
public void runInIgfsThreadPool(Runnable r) {
try {
- igfsSvc.submit(r);
+ igfsSvc.execute(r);
}
catch (RejectedExecutionException ignored) {
// This exception will happen if network speed is too low and data comes faster
@@ -252,4 +252,4 @@ public class IgfsContext {
return mgr;
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
index e534800..4490a68 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
@@ -17,6 +17,7 @@
package org.apache.ignite.internal.processors.igfs;
+import java.util.concurrent.Executor;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteDataStreamer;
import org.apache.ignite.IgniteException;
@@ -36,6 +37,7 @@ import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadabl
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.IgniteInterruptedCheckedException;
import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
import org.apache.ignite.internal.managers.communication.GridMessageListener;
import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
@@ -325,6 +327,8 @@ public class IgfsDataManager extends IgfsManager {
IgniteInternalFuture<byte[]> fut = dataCachePrj.getAsync(key);
if (secReader != null) {
+ Executor exec = igfsCtx.kernalContext().pools().poolForPolicy(GridIoPolicy.IGFS_POOL);
+
fut = fut.chain(new CX1<IgniteInternalFuture<byte[]>, byte[]>() {
@Override public byte[] applyx(IgniteInternalFuture<byte[]> fut) throws IgniteCheckedException {
byte[] res = fut.get();
@@ -365,7 +369,7 @@ public class IgfsDataManager extends IgfsManager {
return res;
}
- });
+ }, exec);
}
else
igfsCtx.metrics().addReadBlocks(1, 0);
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
index ab4ee85..6b23e80 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
@@ -326,7 +326,7 @@ public final class IgfsImpl implements IgfsEx {
// Submit it to the thread pool immediately.
assert dualPool != null;
- dualPool.submit(batch);
+ dualPool.execute(batch);
// Spin in case another batch is currently running.
while (true) {
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java
index 9388a8e..7cba9bb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java
@@ -139,6 +139,7 @@ public class OdbcProcessor extends GridProcessorAdapter {
.logger(log)
.selectorCount(DFLT_SELECTOR_CNT)
.gridName(ctx.gridName())
+ .serverName("odbc")
.tcpNoDelay(DFLT_TCP_NODELAY)
.directBuffer(DFLT_TCP_DIRECT_BUF)
.byteOrder(ByteOrder.nativeOrder())
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
index fd1c2d4..9d9a4d2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
@@ -17,6 +17,7 @@
package org.apache.ignite.internal.processors.platform.compute;
+import java.util.concurrent.Executor;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteCompute;
import org.apache.ignite.binary.BinaryObject;
@@ -410,6 +411,11 @@ public class PlatformCompute extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
+ @Override public IgniteInternalFuture chain(IgniteClosure doneCb, Executor exec) {
+ throw new UnsupportedOperationException("Chain operation is not supported.");
+ }
+
+ /** {@inheritDoc} */
@Override public Throwable error() {
return fut.error();
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java
index b403654..71eca65 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java
@@ -21,7 +21,6 @@ import java.util.HashMap;
import java.util.Map;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.internal.GridKernalContext;
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.processors.rest.GridRestCommand;
import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler;
@@ -38,8 +37,6 @@ import org.apache.ignite.internal.util.nio.GridNioSession;
import org.apache.ignite.internal.util.typedef.C2;
import org.apache.ignite.internal.util.typedef.CIX1;
import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.marshaller.Marshaller;
-import org.apache.ignite.marshaller.jdk.JdkMarshaller;
import org.jetbrains.annotations.Nullable;
import static org.apache.ignite.internal.processors.rest.GridRestCommand.ATOMIC_DECREMENT;
@@ -72,24 +69,16 @@ public class GridTcpMemcachedNioListener extends GridNioServerListenerAdapter<Gr
/** Handler. */
private final GridRestProtocolHandler hnd;
- /** JDK marshaller. */
- private final Marshaller jdkMarshaller = new JdkMarshaller();
-
- /** Context. */
- private final GridKernalContext ctx;
-
/**
* Creates listener which will convert incoming tcp packets to rest requests and forward them to
* a given rest handler.
*
* @param log Logger to use.
* @param hnd Rest handler.
- * @param ctx Context.
*/
- public GridTcpMemcachedNioListener(IgniteLogger log, GridRestProtocolHandler hnd, GridKernalContext ctx) {
+ public GridTcpMemcachedNioListener(IgniteLogger log, GridRestProtocolHandler hnd) {
this.log = log;
this.hnd = hnd;
- this.ctx = ctx;
}
/** {@inheritDoc} */
@@ -462,4 +451,4 @@ public class GridTcpMemcachedNioListener extends GridNioServerListenerAdapter<Gr
return new GridTuple3<>(cmd, quiet, retKey);
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
index 1c1c6dc..3ba6d8e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
@@ -145,7 +145,7 @@ public class GridTcpRestNioListener extends GridNioServerListenerAdapter<GridCli
*/
public GridTcpRestNioListener(IgniteLogger log, GridTcpRestProtocol proto, GridRestProtocolHandler hnd,
GridKernalContext ctx) {
- memcachedLsnr = new GridTcpMemcachedNioListener(log, hnd, ctx);
+ memcachedLsnr = new GridTcpMemcachedNioListener(log, hnd);
redisLsnr = new GridRedisNioListener(log, hnd, ctx);
this.log = log;
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
index 6338fcc..2a002a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
@@ -257,6 +257,7 @@ public class GridTcpRestProtocol extends GridRestProtocolAdapter {
.logger(log)
.selectorCount(cfg.getSelectorCount())
.gridName(ctx.gridName())
+ .serverName("tcp-rest")
.tcpNoDelay(cfg.isNoDelay())
.directBuffer(cfg.isDirectBuffer())
.byteOrder(ByteOrder.nativeOrder())
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index b9b92b8..9a5f077 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -1131,7 +1131,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
// Start service in its own thread.
final ExecutorService exe = svcCtx.executor();
- exe.submit(new Runnable() {
+ exe.execute(new Runnable() {
@Override public void run() {
try {
svc.execute(svcCtx);
@@ -1394,7 +1394,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
return;
try {
- depExe.submit(new BusyRunnable() {
+ depExe.execute(new BusyRunnable() {
@Override public void run0() {
onSystemCacheUpdated(deps);
}
@@ -1587,7 +1587,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
else
topVer = new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion(), 0);
- depExe.submit(new BusyRunnable() {
+ depExe.execute(new BusyRunnable() {
@Override public void run0() {
ClusterNode oldest = ctx.discovery().oldestAliveCacheServerNode(topVer);
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
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 e1937bb..3dfb3c6 100644
--- 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
@@ -153,7 +153,6 @@ import org.apache.ignite.IgniteException;
import org.apache.ignite.IgniteInterruptedException;
import org.apache.ignite.IgniteLogger;
import org.apache.ignite.IgniteSystemProperties;
-import org.apache.ignite.binary.BinaryObject;
import org.apache.ignite.binary.BinaryRawReader;
import org.apache.ignite.binary.BinaryRawWriter;
import org.apache.ignite.cluster.ClusterGroupEmptyException;
@@ -506,10 +505,27 @@ public abstract class IgniteUtils {
}
};
- /**
- * Initializes enterprise check.
+ /** */
+ private static final boolean assertionsEnabled;
+
+ /*
+ *
*/
static {
+ boolean assertionsEnabled0 = true;
+
+ try {
+ assert false;
+
+ assertionsEnabled0 = false;
+ }
+ catch (AssertionError ignored) {
+ assertionsEnabled0 = true;
+ }
+ finally {
+ assertionsEnabled = assertionsEnabled0;
+ }
+
String osName = System.getProperty("os.name");
String osLow = osName.toLowerCase();
@@ -1284,6 +1300,27 @@ public abstract class IgniteUtils {
}
/**
+ * @param threadId Thread ID.
+ * @param sb Builder.
+ */
+ public static void printStackTrace(long threadId, GridStringBuilder sb) {
+ ThreadMXBean mxBean = ManagementFactory.getThreadMXBean();
+
+ ThreadInfo threadInfo = mxBean.getThreadInfo(threadId, Integer.MAX_VALUE);
+
+ printThreadInfo(threadInfo, sb, Collections.<Long>emptySet());
+ }
+
+ /**
+ * @return {@code true} if there is java level deadlock.
+ */
+ public static boolean deadlockPresent() {
+ ThreadMXBean mxBean = ManagementFactory.getThreadMXBean();
+
+ return !F.isEmpty(mxBean.findDeadlockedThreads());
+ }
+
+ /**
* Prints single thread info to a buffer.
*
* @param threadInfo Thread info.
@@ -6141,6 +6178,13 @@ public abstract class IgniteUtils {
}
/**
+ * @return {@code True} if assertions enabled.
+ */
+ public static boolean assertionsEnabled() {
+ return assertionsEnabled;
+ }
+
+ /**
* Gets OS JDK string.
*
* @return OS JDK string.
@@ -8337,6 +8381,18 @@ public abstract class IgniteUtils {
}
/**
+ * Gets absolute value for long. If argument is {@link Long#MIN_VALUE}, then {@code 0} is returned.
+ *
+ * @param i Argument.
+ * @return Absolute value.
+ */
+ public static long safeAbs(long i) {
+ i = Math.abs(i);
+
+ return i < 0 ? 0 : i;
+ }
+
+ /**
* Gets wrapper class for a primitive type.
*
* @param cls Class. If {@code null}, method is no-op.
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java
new file mode 100644
index 0000000..e9ec74b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java
@@ -0,0 +1,667 @@
+/*
+ * 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.util;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.locks.LockSupport;
+import org.apache.ignite.IgniteInterruptedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Striped executor.
+ */
+public class StripedExecutor implements ExecutorService {
+ /** Stripes. */
+ private final Stripe[] stripes;
+
+ /** For starvation checks. */
+ private final long[] completedCntrs;
+
+ /** */
+ private final IgniteLogger log;
+
+ /**
+ * Constructor.
+ *
+ * @param cnt Count.
+ */
+ public StripedExecutor(int cnt, String gridName, String poolName, final IgniteLogger log) {
+ A.ensure(cnt > 0, "cnt > 0");
+
+ boolean success = false;
+
+ stripes = new Stripe[cnt];
+
+ completedCntrs = new long[cnt];
+
+ Arrays.fill(completedCntrs, -1);
+
+ this.log = log;
+
+ try {
+ for (int i = 0; i < cnt; i++) {
+ stripes[i] = new StripeConcurrentQueue(
+ gridName,
+ poolName,
+ i,
+ log);
+
+ stripes[i].start();
+ }
+
+ success = true;
+ }
+ catch (Error | RuntimeException e) {
+ U.error(log, "Failed to initialize striped pool.", e);
+
+ throw e;
+ }
+ finally {
+ if (!success) {
+ for (Stripe stripe : stripes) {
+ if (stripe != null)
+ stripe.signalStop();
+ }
+
+ for (Stripe stripe : stripes) {
+ if (stripe != null)
+ stripe.awaitStop();
+ }
+ }
+ }
+ }
+
+ /**
+ * Checks starvation in striped pool. Maybe too verbose
+ * but this is needed to faster debug possible issues.
+ */
+ public void checkStarvation() {
+ for (int i = 0; i < stripes.length; i++) {
+ Stripe stripe = stripes[i];
+
+ long completedCnt = stripe.completedCnt;
+
+ boolean active = stripe.active;
+
+ if (completedCntrs[i] != -1 &&
+ completedCntrs[i] == completedCnt &&
+ active) {
+ boolean deadlockPresent = U.deadlockPresent();
+
+ GridStringBuilder sb = new GridStringBuilder();
+
+ sb.a(">>> Possible starvation in striped pool: ")
+ .a(stripe.thread.getName()).a(U.nl())
+ .a(stripe.queueToString()).a(U.nl())
+ .a("deadlock: ").a(deadlockPresent).a(U.nl())
+ .a("completed: ").a(completedCnt).a(U.nl());
+
+ U.printStackTrace(
+ stripe.thread.getId(),
+ sb);
+
+ String msg = sb.toString();
+
+ U.warn(log, msg);
+ }
+
+ if (active || completedCnt > 0)
+ completedCntrs[i] = completedCnt;
+ }
+ }
+
+ /**
+ * @return Stripes count.
+ */
+ public int stripes() {
+ return stripes.length;
+ }
+
+ /**
+ * Execute command.
+ *
+ * @param idx Index.
+ * @param cmd Command.
+ */
+ public void execute(int idx, Runnable cmd) {
+ if (idx == -1)
+ execute(cmd);
+ else {
+ assert idx >= 0 : idx;
+
+ stripes[idx % stripes.length].execute(cmd);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public void shutdown() {
+ signalStop();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void execute(@NotNull Runnable cmd) {
+ stripes[ThreadLocalRandom.current().nextInt(stripes.length)].execute(cmd);
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * @return Empty list (always).
+ */
+ @NotNull @Override public List<Runnable> shutdownNow() {
+ signalStop();
+
+ return Collections.emptyList();
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean awaitTermination(
+ long timeout,
+ @NotNull TimeUnit unit
+ ) throws InterruptedException {
+ awaitStop();
+
+ return true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean isShutdown() {
+ for (Stripe stripe : stripes) {
+ if (stripe != null && stripe.stopping)
+ return true;
+ }
+
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean isTerminated() {
+ for (Stripe stripe : stripes) {
+ if (stripe.thread.getState() != Thread.State.TERMINATED)
+ return false;
+ }
+
+ return true;
+ }
+
+ /**
+ * Stops executor.
+ */
+ public void stop() {
+ signalStop();
+
+ awaitStop();
+ }
+
+ /**
+ * Signals all stripes.
+ */
+ private void signalStop() {
+ for (Stripe stripe : stripes)
+ stripe.signalStop();
+ }
+
+ /**
+ * @throws IgniteInterruptedException If interrupted.
+ */
+ private void awaitStop() throws IgniteInterruptedException {
+ for (Stripe stripe : stripes)
+ stripe.awaitStop();
+ }
+
+ /**
+ * @return Return total queue size of all stripes.
+ */
+ public int queueSize() {
+ int size = 0;
+
+ for (Stripe stripe : stripes)
+ size += stripe.queueSize();
+
+ return size;
+ }
+
+ /**
+ * @return Completed tasks count.
+ */
+ public long completedTasks() {
+ long cnt = 0;
+
+ for (Stripe stripe : stripes)
+ cnt += stripe.completedCnt;
+
+ return cnt;
+ }
+
+ /**
+ * Operation not supported.
+ */
+ @NotNull @Override public <T> Future<T> submit(
+ @NotNull Runnable task,
+ T res
+ ) {
+ throw new UnsupportedOperationException();
+ }
+
+ /**
+ * Operation not supported.
+ */
+ @NotNull @Override public Future<?> submit(@NotNull Runnable task) {
+ throw new UnsupportedOperationException();
+ }
+
+ /**
+ * Operation not supported.
+ */
+ @NotNull @Override public <T> Future<T> submit(@NotNull Callable<T> task) {
+ throw new UnsupportedOperationException();
+ }
+
+ /**
+ * Operation not supported.
+ */
+ @NotNull @Override public <T> List<Future<T>> invokeAll(@NotNull Collection<? extends Callable<T>> tasks)
+ throws InterruptedException {
+ throw new UnsupportedOperationException();
+ }
+
+ /**
+ * Operation not supported.
+ */
+ @NotNull @Override public <T> List<Future<T>> invokeAll(
+ @NotNull Collection<? extends Callable<T>> tasks,
+ long timeout,
+ @NotNull TimeUnit unit
+ ) throws InterruptedException {
+ throw new UnsupportedOperationException();
+ }
+
+ /**
+ * Operation not supported.
+ */
+ @NotNull @Override public <T> T invokeAny(@NotNull Collection<? extends Callable<T>> tasks)
+ throws InterruptedException, ExecutionException {
+ throw new UnsupportedOperationException();
+ }
+
+ /**
+ * Operation not supported.
+ */
+ @Override public <T> T invokeAny(
+ @NotNull Collection<? extends Callable<T>> tasks,
+ long timeout,
+ @NotNull TimeUnit unit
+ ) throws InterruptedException, ExecutionException, TimeoutException {
+ throw new UnsupportedOperationException();
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(StripedExecutor.class, this);
+ }
+
+ /**
+ * Stripe.
+ */
+ private static abstract class Stripe implements Runnable {
+ /** */
+ private final String gridName;
+
+ /** */
+ private final String poolName;
+
+ /** */
+ private final int idx;
+
+ /** */
+ private final IgniteLogger log;
+
+ /** Stopping flag. */
+ private volatile boolean stopping;
+
+ /** */
+ private volatile long completedCnt;
+
+ /** */
+ private volatile boolean active;
+
+ /** Thread executing the loop. */
+ protected Thread thread;
+
+ /**
+ * @param gridName Grid name.
+ * @param poolName Pool name.
+ * @param idx Stripe index.
+ * @param log Logger.
+ */
+ public Stripe(
+ String gridName,
+ String poolName,
+ int idx,
+ IgniteLogger log
+ ) {
+ this.gridName = gridName;
+ this.poolName = poolName;
+ this.idx = idx;
+ this.log = log;
+ }
+
+ /**
+ * Starts the stripe.
+ */
+ void start() {
+ thread = new IgniteThread(gridName, poolName + "-stripe-" + idx, this);
+
+ thread.start();
+ }
+
+ /**
+ * Stop the stripe.
+ */
+ void signalStop() {
+ stopping = true;
+
+ U.interrupt(thread);
+ }
+
+ /**
+ * Await thread stop.
+ */
+ void awaitStop() {
+ try {
+ if (thread != null)
+ thread.join();
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+
+ throw new IgniteInterruptedException(e);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public void run() {
+ while (!stopping) {
+ Runnable cmd;
+
+ try {
+ cmd = take();
+
+ if (cmd != null) {
+ active = true;
+
+ try {
+ cmd.run();
+ }
+ finally {
+ active = false;
+ completedCnt++;
+ }
+ }
+ }
+ catch (InterruptedException e) {
+ stopping = true;
+
+ Thread.currentThread().interrupt();
+
+ return;
+ }
+ catch (Throwable e) {
+ U.error(log, "Failed to execute runnable.", e);
+ }
+ }
+ }
+
+ /**
+ * Execute the command.
+ *
+ * @param cmd Command.
+ */
+ abstract void execute(Runnable cmd);
+
+ /**
+ * @return Next runnable.
+ * @throws InterruptedException If interrupted.
+ */
+ abstract Runnable take() throws InterruptedException;
+
+ /**
+ * @return Queue size.
+ */
+ abstract int queueSize();
+
+ /**
+ * @return Stripe's queue to string presentation.
+ */
+ abstract String queueToString();
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(Stripe.class, this);
+ }
+ }
+
+ /**
+ * Stripe.
+ */
+ private static class StripeConcurrentQueue extends Stripe {
+ /** Queue. */
+ private final Queue<Runnable> queue = new ConcurrentLinkedQueue<>();
+
+ /** */
+ private volatile boolean parked;
+
+ /**
+ * @param gridName Grid name.
+ * @param poolName Pool name.
+ * @param idx Stripe index.
+ * @param log Logger.
+ */
+ public StripeConcurrentQueue(
+ String gridName,
+ String poolName,
+ int idx,
+ IgniteLogger log
+ ) {
+ super(gridName,
+ poolName,
+ idx,
+ log);
+ }
+
+ /** {@inheritDoc} */
+ @Override Runnable take() throws InterruptedException {
+ Runnable r;
+
+ for (int i = 0; i < 2048; i++) {
+ r = queue.poll();
+
+ if (r != null)
+ return r;
+ }
+
+ parked = true;
+
+ try {
+ for (;;) {
+ r = queue.poll();
+
+ if (r != null)
+ return r;
+
+ LockSupport.park();
+
+ if (Thread.interrupted())
+ throw new InterruptedException();
+ }
+ }
+ finally {
+ parked = false;
+ }
+ }
+
+ /** {@inheritDoc} */
+ void execute(Runnable cmd) {
+ queue.add(cmd);
+
+ if (parked)
+ LockSupport.unpark(thread);
+ }
+
+ /** {@inheritDoc} */
+ @Override String queueToString() {
+ return String.valueOf(queue);
+ }
+
+ /** {@inheritDoc} */
+ @Override int queueSize() {
+ return queue.size();
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(StripeConcurrentQueue.class, this, super.toString());
+ }
+ }
+
+ /**
+ * Stripe.
+ */
+ private static class StripeConcurrentQueueNoPark extends Stripe {
+ /** Queue. */
+ private final Queue<Runnable> queue = new ConcurrentLinkedQueue<>();
+
+ /**
+ * @param gridName Grid name.
+ * @param poolName Pool name.
+ * @param idx Stripe index.
+ * @param log Logger.
+ */
+ public StripeConcurrentQueueNoPark(
+ String gridName,
+ String poolName,
+ int idx,
+ IgniteLogger log
+ ) {
+ super(gridName,
+ poolName,
+ idx,
+ log);
+ }
+
+ /** {@inheritDoc} */
+ @Override Runnable take() {
+ for (;;) {
+ Runnable r = queue.poll();
+
+ if (r != null)
+ return r;
+ }
+ }
+
+ /** {@inheritDoc} */
+ void execute(Runnable cmd) {
+ queue.add(cmd);
+ }
+
+ /** {@inheritDoc} */
+ @Override int queueSize() {
+ return queue.size();
+ }
+
+ /** {@inheritDoc} */
+ @Override String queueToString() {
+ return String.valueOf(queue);
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(StripeConcurrentQueueNoPark.class, this, super.toString());
+ }
+ }
+
+ /**
+ * Stripe.
+ */
+ private static class StripeConcurrentBlockingQueue extends Stripe {
+ /** Queue. */
+ private final BlockingQueue<Runnable> queue = new LinkedBlockingQueue<>();
+
+ /**
+ * @param gridName Grid name.
+ * @param poolName Pool name.
+ * @param idx Stripe index.
+ * @param log Logger.
+ */
+ public StripeConcurrentBlockingQueue(
+ String gridName,
+ String poolName,
+ int idx,
+ IgniteLogger log
+ ) {
+ super(gridName,
+ poolName,
+ idx,
+ log);
+ }
+
+ /** {@inheritDoc} */
+ @Override Runnable take() throws InterruptedException {
+ return queue.take();
+ }
+
+ /** {@inheritDoc} */
+ void execute(Runnable cmd) {
+ queue.add(cmd);
+ }
+
+ /** {@inheritDoc} */
+ @Override int queueSize() {
+ return queue.size();
+ }
+
+ /** {@inheritDoc} */
+ @Override String queueToString() {
+ return String.valueOf(queue);
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(StripeConcurrentBlockingQueue.class, this, super.toString());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java
index 6baedbd..dc63adc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java
@@ -17,6 +17,7 @@
package org.apache.ignite.internal.util.future;
+import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.internal.IgniteInternalFuture;
@@ -152,6 +153,29 @@ public class GridFinishedFuture<T> implements IgniteInternalFuture<T> {
}
/** {@inheritDoc} */
+ @Override public <T1> IgniteInternalFuture<T1> chain(final IgniteClosure<? super IgniteInternalFuture<T>, T1> doneCb, Executor exec) {
+ final GridFutureAdapter<T1> fut = new GridFutureAdapter<>();
+
+ exec.execute(new Runnable() {
+ @Override public void run() {
+ try {
+ fut.onDone(doneCb.apply(GridFinishedFuture.this));
+ }
+ catch (GridClosureException e) {
+ fut.onDone(e.unwrap());
+ }
+ catch (RuntimeException | Error e) {
+ fut.onDone(e);
+
+ throw e;
+ }
+ }
+ });
+
+ return fut;
+ }
+
+ /** {@inheritDoc} */
@Override public String toString() {
return S.toString(GridFinishedFuture.class, this);
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
index 2cd534e..c8d85cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
@@ -18,6 +18,7 @@
package org.apache.ignite.internal.util.future;
import java.util.Arrays;
+import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.AbstractQueuedSynchronizer;
import org.apache.ignite.IgniteCheckedException;
@@ -229,7 +230,13 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
/** {@inheritDoc} */
@Override public <T> IgniteInternalFuture<T> chain(final IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb) {
- return new ChainFuture<>(this, doneCb);
+ return new ChainFuture<>(this, doneCb, null);
+ }
+
+ /** {@inheritDoc} */
+ @Override public <T> IgniteInternalFuture<T> chain(final IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb,
+ Executor exec) {
+ return new ChainFuture<>(this, doneCb, exec);
}
/**
@@ -487,15 +494,17 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
/**
* @param fut Future.
* @param doneCb Closure.
+ * @param cbExec Optional executor to run callback.
*/
ChainFuture(
GridFutureAdapter<R> fut,
- IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb
+ IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb,
+ @Nullable Executor cbExec
) {
this.fut = fut;
this.doneCb = doneCb;
- fut.listen(new GridFutureChainListener<>(this, doneCb));
+ fut.listen(new GridFutureChainListener<>(this, doneCb, cbExec));
}
/** {@inheritDoc} */
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureChainListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureChainListener.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureChainListener.java
index 947b2ad..367f5d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureChainListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureChainListener.java
@@ -17,15 +17,17 @@
package org.apache.ignite.internal.util.future;
+import java.util.concurrent.Executor;
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.util.lang.GridClosureException;
import org.apache.ignite.lang.IgniteClosure;
import org.apache.ignite.lang.IgniteInClosure;
+import org.jetbrains.annotations.Nullable;
/**
* Future listener to fill chained future with converted result of the source future.
*/
-public class GridFutureChainListener<T, R> implements IgniteInClosure<IgniteInternalFuture<T>> {
+class GridFutureChainListener<T, R> implements IgniteInClosure<IgniteInternalFuture<T>> {
/** */
private static final long serialVersionUID = 0L;
@@ -35,21 +37,43 @@ public class GridFutureChainListener<T, R> implements IgniteInClosure<IgniteInte
/** Done callback. */
private final IgniteClosure<? super IgniteInternalFuture<T>, R> doneCb;
+ /** */
+ private Executor cbExec;
+
/**
* Constructs chain listener.
+ *
* @param fut Target future.
* @param doneCb Done callback.
+ * @param cbExec Optional executor to run callback.
*/
public GridFutureChainListener(
GridFutureAdapter<R> fut,
- IgniteClosure<? super IgniteInternalFuture<T>, R> doneCb
+ IgniteClosure<? super IgniteInternalFuture<T>, R> doneCb,
+ @Nullable Executor cbExec
) {
this.fut = fut;
this.doneCb = doneCb;
+ this.cbExec = cbExec;
}
/** {@inheritDoc} */
- @Override public void apply(IgniteInternalFuture<T> t) {
+ @Override public void apply(final IgniteInternalFuture<T> t) {
+ if (cbExec != null) {
+ cbExec.execute(new Runnable() {
+ @Override public void run() {
+ applyCallback(t);
+ }
+ });
+ }
+ else
+ applyCallback(t);
+ }
+
+ /**
+ * @param t Target future.
+ */
+ private void applyCallback(IgniteInternalFuture<T> t) {
try {
fut.onDone(doneCb.apply(t));
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/IpcToNioAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/IpcToNioAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/IpcToNioAdapter.java
index 6820dc7..d108b56 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/IpcToNioAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/IpcToNioAdapter.java
@@ -201,7 +201,7 @@ public class IpcToNioAdapter<T> {
}
/** {@inheritDoc} */
- @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) {
+ @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) {
assert ses == IpcToNioAdapter.this.ses;
return send((Message)msg);
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridAbstractCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridAbstractCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridAbstractCommunicationClient.java
index 9b014ec..f2ab932 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridAbstractCommunicationClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridAbstractCommunicationClient.java
@@ -35,14 +35,24 @@ public abstract class GridAbstractCommunicationClient implements GridCommunicati
/** Metrics listener. */
protected final GridNioMetricsListener metricsLsnr;
+ /** */
+ private final int connIdx;
+
/**
+ * @param connIdx Connection index.
* @param metricsLsnr Metrics listener.
*/
- protected GridAbstractCommunicationClient(@Nullable GridNioMetricsListener metricsLsnr) {
+ protected GridAbstractCommunicationClient(int connIdx, @Nullable GridNioMetricsListener metricsLsnr) {
+ this.connIdx = connIdx;
this.metricsLsnr = metricsLsnr;
}
/** {@inheritDoc} */
+ @Override public int connectionIndex() {
+ return connIdx;
+ }
+
+ /** {@inheritDoc} */
@Override public boolean close() {
return reserves.compareAndSet(0, -1);
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java
index 0de54e9..71b2c24 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java
@@ -96,15 +96,20 @@ public interface GridCommunicationClient {
/**
* @param nodeId Remote node ID. Provided only for sync clients.
* @param msg Message to send.
- * @param closure Ack closure.
+ * @param c Ack closure.
* @throws IgniteCheckedException If failed.
* @return {@code True} if should try to resend message.
*/
- public boolean sendMessage(@Nullable UUID nodeId, Message msg, @Nullable IgniteInClosure<IgniteException> closure)
+ public boolean sendMessage(@Nullable UUID nodeId, Message msg, @Nullable IgniteInClosure<IgniteException> c)
throws IgniteCheckedException;
/**
* @return {@code True} if send is asynchronous.
*/
public boolean async();
+
+ /**
+ * @return Connection index.
+ */
+ public int connectionIndex();
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
index 213fd8d..7987d3d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
@@ -62,13 +62,20 @@ public class GridConnectionBytesVerifyFilter extends GridNioFilterAdapter {
}
/** {@inheritDoc} */
- @Override public void onExceptionCaught(GridNioSession ses, IgniteCheckedException ex) throws IgniteCheckedException {
+ @Override public void onExceptionCaught(
+ GridNioSession ses,
+ IgniteCheckedException ex
+ ) throws IgniteCheckedException {
proceedExceptionCaught(ses, ex);
}
/** {@inheritDoc} */
- @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
- return proceedSessionWrite(ses, msg);
+ @Override public GridNioFuture<?> onSessionWrite(
+ GridNioSession ses,
+ Object msg,
+ boolean fut
+ ) throws IgniteCheckedException {
+ return proceedSessionWrite(ses, msg, fut);
}
/** {@inheritDoc} */
@@ -137,4 +144,4 @@ public class GridConnectionBytesVerifyFilter extends GridNioFilterAdapter {
@Override public void onSessionWriteTimeout(GridNioSession ses) throws IgniteCheckedException {
proceedSessionWriteTimeout(ses);
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioAsyncNotifyFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioAsyncNotifyFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioAsyncNotifyFilter.java
index 9925d2e..40c87cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioAsyncNotifyFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioAsyncNotifyFilter.java
@@ -107,8 +107,12 @@ public class GridNioAsyncNotifyFilter extends GridNioFilterAdapter {
}
/** {@inheritDoc} */
- @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
- return proceedSessionWrite(ses, msg);
+ @Override public GridNioFuture<?> onSessionWrite(
+ GridNioSession ses,
+ Object msg,
+ boolean fut
+ ) throws IgniteCheckedException {
+ return proceedSessionWrite(ses, msg, fut);
}
/** {@inheritDoc} */
@@ -139,4 +143,4 @@ public class GridNioAsyncNotifyFilter extends GridNioFilterAdapter {
"originalEx=" + ex + ", ex=" + e + ']');
}
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
index 7083ccf..343e625 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
@@ -71,20 +71,27 @@ public class GridNioCodecFilter extends GridNioFilterAdapter {
}
/** {@inheritDoc} */
- @Override public void onExceptionCaught(GridNioSession ses, IgniteCheckedException ex) throws IgniteCheckedException {
+ @Override public void onExceptionCaught(
+ GridNioSession ses,
+ IgniteCheckedException ex
+ ) throws IgniteCheckedException {
proceedExceptionCaught(ses, ex);
}
/** {@inheritDoc} */
- @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
+ @Override public GridNioFuture<?> onSessionWrite(
+ GridNioSession ses,
+ Object msg,
+ boolean fut
+ ) throws IgniteCheckedException {
// No encoding needed in direct mode.
if (directMode)
- return proceedSessionWrite(ses, msg);
+ return proceedSessionWrite(ses, msg, fut);
try {
ByteBuffer res = parser.encode(ses, msg);
- return proceedSessionWrite(ses, res);
+ return proceedSessionWrite(ses, res, fut);
}
catch (IOException e) {
throw new GridNioException(e);
@@ -137,4 +144,4 @@ public class GridNioCodecFilter extends GridNioFilterAdapter {
@Override public void onSessionWriteTimeout(GridNioSession ses) throws IgniteCheckedException {
proceedSessionWriteTimeout(ses);
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilter.java
index 5f88b1f..f7928c4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilter.java
@@ -105,10 +105,15 @@ public interface GridNioFilter {
*
* @param ses Session instance.
* @param msg Message to send.
- * @return Write future.
+ * @param fut {@code True} if write future should be created.
+ * @return Write future or {@code null}.
* @throws IgniteCheckedException If filter is not in chain or GridNioException occurred in the underlying filter.
*/
- public GridNioFuture<?> proceedSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException;
+ public GridNioFuture<?> proceedSessionWrite(
+ GridNioSession ses,
+ Object msg,
+ boolean fut
+ ) throws IgniteCheckedException;
/**
* Forwards session close request to the next logical filter in filter chain.
@@ -149,10 +154,11 @@ public interface GridNioFilter {
*
* @param ses Session on which message should be written.
* @param msg Message being written.
- * @return Write future.
+ * @param fut {@code True} if write future should be created.
+ * @return Write future or {@code null}.
* @throws GridNioException If GridNioException occurred while handling event.
*/
- public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException;
+ public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) throws IgniteCheckedException;
/**
* Invoked when a new messages received.
@@ -241,4 +247,4 @@ public interface GridNioFilter {
* @throws IgniteCheckedException If filter is not in chain or GridNioException occurred in the underlying filter.
*/
public GridNioFuture<?> onResumeReads(GridNioSession ses) throws IgniteCheckedException;
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterAdapter.java
index 18ab1b2..58ddae5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterAdapter.java
@@ -108,10 +108,14 @@ public abstract class GridNioFilterAdapter implements GridNioFilter {
}
/** {@inheritDoc} */
- @Override public GridNioFuture<?> proceedSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
+ @Override public GridNioFuture<?> proceedSessionWrite(
+ GridNioSession ses,
+ Object msg,
+ boolean fut
+ ) throws IgniteCheckedException {
checkNext();
- return nextFilter.onSessionWrite(ses, msg);
+ return nextFilter.onSessionWrite(ses, msg, fut);
}
/** {@inheritDoc} */
@@ -180,4 +184,4 @@ public abstract class GridNioFilterAdapter implements GridNioFilter {
throw new GridNioException("Failed to proceed with filter call since previous filter is not set " +
"(do you use filter outside the filter chain?): " + getClass().getName());
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
index a3a74e3..8cc690b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
@@ -181,8 +181,12 @@ public class GridNioFilterChain<T> extends GridNioFilterAdapter {
* @return Send future.
* @throws IgniteCheckedException If IgniteCheckedException occurred while handling event.
*/
- @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
- return tail.onSessionWrite(ses, msg);
+ @Override public GridNioFuture<?> onSessionWrite(
+ GridNioSession ses,
+ Object msg,
+ boolean fut
+ ) throws IgniteCheckedException {
+ return tail.onSessionWrite(ses, msg, fut);
}
/**
@@ -255,9 +259,9 @@ public class GridNioFilterChain<T> extends GridNioFilterAdapter {
}
/** {@inheritDoc} */
- @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg)
+ @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut)
throws IgniteCheckedException {
- return proceedSessionWrite(ses, msg);
+ return proceedSessionWrite(ses, msg, fut);
}
/** {@inheritDoc} */
@@ -290,4 +294,4 @@ public class GridNioFilterChain<T> extends GridNioFilterAdapter {
return proceedResumeReads(ses);
}
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java
index b02acc8..6c0c9c6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java
@@ -45,9 +45,9 @@ public interface GridNioFuture<R> extends IgniteInternalFuture<R> {
/**
* Sets ack closure which will be applied when ack received.
*
- * @param closure Ack closure.
+ * @param c Ack closure.
*/
- public void ackClosure(IgniteInClosure<IgniteException> closure);
+ public void ackClosure(IgniteInClosure<IgniteException> c);
/**
* The method will be called when ack received.
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
index 35480ac..6258c13 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
@@ -35,8 +35,8 @@ public class GridNioRecoveryDescriptor {
/** Number of acknowledged messages. */
private long acked;
- /** Unacknowledged message futures. */
- private final ArrayDeque<GridNioFuture<?>> msgFuts;
+ /** Unacknowledged messages. */
+ private final ArrayDeque<SessionWriteRequest> msgReqs;
/** Number of messages to resend. */
private int resendCnt;
@@ -77,23 +77,40 @@ public class GridNioRecoveryDescriptor {
/** Number of descriptor reservations (for info purposes). */
private int reserveCnt;
+ /** */
+ private final boolean pairedConnections;
+
/**
+ * @param pairedConnections {@code True} if in/out connections pair is used for communication with node.
* @param queueLimit Maximum size of unacknowledged messages queue.
* @param node Node.
* @param log Logger.
*/
- public GridNioRecoveryDescriptor(int queueLimit, ClusterNode node, IgniteLogger log) {
+ public GridNioRecoveryDescriptor(
+ boolean pairedConnections,
+ int queueLimit,
+ ClusterNode node,
+ IgniteLogger log
+ ) {
assert !node.isLocal() : node;
assert queueLimit > 0;
- msgFuts = new ArrayDeque<>(queueLimit);
+ msgReqs = new ArrayDeque<>(queueLimit);
+ this.pairedConnections = pairedConnections;
this.queueLimit = queueLimit;
this.node = node;
this.log = log;
}
/**
+ * @return {@code True} if in/out connections pair is used for communication with node.
+ */
+ public boolean pairedConnections() {
+ return pairedConnections;
+ }
+
+ /**
* @return Connect count.
*/
public long incrementConnectCount() {
@@ -154,19 +171,19 @@ public class GridNioRecoveryDescriptor {
}
/**
- * @param fut NIO future.
+ * @param req Write request.
* @return {@code False} if queue limit is exceeded.
*/
- public boolean add(GridNioFuture<?> fut) {
- assert fut != null;
+ public boolean add(SessionWriteRequest req) {
+ assert req != null;
- if (!fut.skipRecovery()) {
+ if (!req.skipRecovery()) {
if (resendCnt == 0) {
- msgFuts.addLast(fut);
+ msgReqs.addLast(req);
sentCnt++;
- return msgFuts.size() < queueLimit;
+ return msgReqs.size() < queueLimit;
}
else
resendCnt--;
@@ -181,21 +198,19 @@ public class GridNioRecoveryDescriptor {
public void ackReceived(long rcvCnt) {
if (log.isDebugEnabled())
log.debug("Handle acknowledgment [acked=" + acked + ", rcvCnt=" + rcvCnt +
- ", msgFuts=" + msgFuts.size() + ']');
+ ", msgReqs=" + msgReqs.size() + ']');
while (acked < rcvCnt) {
- GridNioFuture<?> fut = msgFuts.pollFirst();
+ SessionWriteRequest req = msgReqs.pollFirst();
- assert fut != null : "Missed message future [rcvCnt=" + rcvCnt +
+ assert req != null : "Missed message [rcvCnt=" + rcvCnt +
", acked=" + acked +
", desc=" + this + ']';
- assert fut.isDone() : fut;
-
- if (fut.ackClosure() != null)
- fut.ackClosure().apply(null);
+ if (req.ackClosure() != null)
+ req.ackClosure().apply(null);
- fut.onAckReceived();
+ req.onAckReceived();
acked++;
}
@@ -214,7 +229,7 @@ public class GridNioRecoveryDescriptor {
* @return {@code False} if descriptor is reserved.
*/
public boolean onNodeLeft() {
- GridNioFuture<?>[] futs = null;
+ SessionWriteRequest[] reqs = null;
synchronized (this) {
nodeLeft = true;
@@ -222,24 +237,24 @@ public class GridNioRecoveryDescriptor {
if (reserved)
return false;
- if (!msgFuts.isEmpty()) {
- futs = msgFuts.toArray(new GridNioFuture<?>[msgFuts.size()]);
+ if (!msgReqs.isEmpty()) {
+ reqs = msgReqs.toArray(new SessionWriteRequest[msgReqs.size()]);
- msgFuts.clear();
+ msgReqs.clear();
}
}
- if (futs != null)
- completeOnNodeLeft(futs);
+ if (reqs != null)
+ notifyOnNodeLeft(reqs);
return true;
}
/**
- * @return Message futures for unacknowledged messages.
+ * @return Requests for unacknowledged messages.
*/
- public Deque<GridNioFuture<?>> messagesFutures() {
- return msgFuts;
+ public Deque<SessionWriteRequest> messagesRequests() {
+ return msgReqs;
}
/**
@@ -277,14 +292,14 @@ public class GridNioRecoveryDescriptor {
if (!nodeLeft)
ackReceived(rcvCnt);
- resendCnt = msgFuts.size();
+ resendCnt = msgReqs.size();
}
}
/**
*
*/
- public void connected() {
+ public void onConnected() {
synchronized (this) {
assert reserved : this;
assert !connected : this;
@@ -306,10 +321,37 @@ public class GridNioRecoveryDescriptor {
}
/**
+ * @return Connected flag.
+ */
+ public boolean connected() {
+ synchronized (this) {
+ return connected;
+ }
+ }
+
+ /**
+ * @return Reserved flag.
+ */
+ public boolean reserved() {
+ synchronized (this) {
+ return reserved;
+ }
+ }
+
+ /**
+ * @return Current handshake index.
+ */
+ public Long handshakeIndex() {
+ synchronized (this) {
+ return handshakeReq != null ? handshakeReq.get1() : null;
+ }
+ }
+
+ /**
*
*/
public void release() {
- GridNioFuture<?>[] futs = null;
+ SessionWriteRequest[] futs = null;
synchronized (this) {
connected = false;
@@ -329,15 +371,15 @@ public class GridNioRecoveryDescriptor {
notifyAll();
}
- if (nodeLeft && !msgFuts.isEmpty()) {
- futs = msgFuts.toArray(new GridNioFuture<?>[msgFuts.size()]);
+ if (nodeLeft && !msgReqs.isEmpty()) {
+ futs = msgReqs.toArray(new SessionWriteRequest[msgReqs.size()]);
- msgFuts.clear();
+ msgReqs.clear();
}
}
if (futs != null)
- completeOnNodeLeft(futs);
+ notifyOnNodeLeft(futs);
}
/**
@@ -398,16 +440,16 @@ public class GridNioRecoveryDescriptor {
}
/**
- * @param futs Futures to complete.
+ * @param reqs Requests to notify about error.
*/
- private void completeOnNodeLeft(GridNioFuture<?>[] futs) {
- for (GridNioFuture<?> msg : futs) {
- IOException e = new IOException("Failed to send message, node has left: " + node.id());
+ private void notifyOnNodeLeft(SessionWriteRequest[] reqs) {
+ IOException e = new IOException("Failed to send message, node has left: " + node.id());
- ((GridNioFutureImpl)msg).onDone(e);
+ for (SessionWriteRequest req : reqs) {
+ req.onError(e);
- if (msg.ackClosure() != null)
- msg.ackClosure().apply(new IgniteException(e));
+ if (req.ackClosure() != null)
+ req.ackClosure().apply(new IgniteException(e));
}
}
[24/25] ignite git commit: ignite-3699: CreatedExpiryPolicy doesn't
work if entry is loaded from store (partial fix)
Posted by sb...@apache.org.
ignite-3699: CreatedExpiryPolicy doesn't work if entry is loaded from store (partial fix)
Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/27a81c86
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/27a81c86
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/27a81c86
Branch: refs/heads/ignite-4371
Commit: 27a81c865f6574f6c287dad6a401171bf0a9c53a
Parents: fa358cc
Author: Amir Akhmedov <am...@gmail.com>
Authored: Fri Dec 9 14:01:10 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 9 14:01:10 2016 +0300
----------------------------------------------------------------------
.../processors/cache/GridCacheAdapter.java | 7 +++-
.../processors/cache/GridCacheEntryEx.java | 4 +-
.../processors/cache/GridCacheMapEntry.java | 17 +++++++--
.../distributed/dht/GridDhtLockFuture.java | 23 ++++++++++-
.../dht/atomic/GridDhtAtomicCache.java | 2 +-
.../transactions/IgniteTxLocalAdapter.java | 2 +-
.../processors/cache/GridCacheTestEntryEx.java | 3 +-
...eCacheExpiryPolicyWithStoreAbstractTest.java | 40 ++++++++++++++++++++
8 files changed, 87 insertions(+), 11 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 1db3931..88aa4e0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -1882,7 +1882,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
@Nullable final UUID subjId,
final String taskName,
final boolean deserializeBinary,
- @Nullable IgniteCacheExpiryPolicy expiry,
+ @Nullable final IgniteCacheExpiryPolicy expiry,
final boolean skipVals,
final boolean keepCacheObjects,
boolean canRemap,
@@ -2017,7 +2017,10 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
GridCacheEntryEx entry = entryEx(key);
try {
- GridCacheVersion verSet = entry.versionedValue(cacheVal, ver, null);
+ GridCacheVersion verSet = entry.versionedValue(cacheVal,
+ ver,
+ null,
+ expiry);
boolean set = verSet != null;
http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
index d8194fc..9e9b496 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
@@ -725,13 +725,15 @@ public interface GridCacheEntryEx {
* @param val New value.
* @param curVer Version to match or {@code null} if match is not required.
* @param newVer Version to set.
+ * @param loadExpiryPlc Expiry policy if entry is loaded from store.
* @return Non null version if value was set.
* @throws IgniteCheckedException If index could not be updated.
* @throws GridCacheEntryRemovedException If entry was removed.
*/
public GridCacheVersion versionedValue(CacheObject val,
@Nullable GridCacheVersion curVer,
- @Nullable GridCacheVersion newVer)
+ @Nullable GridCacheVersion newVer,
+ @Nullable IgniteCacheExpiryPolicy loadExpiryPlc)
throws IgniteCheckedException, GridCacheEntryRemovedException;
/**
http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 31baeda..52b779d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -3551,7 +3551,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
/** {@inheritDoc} */
@Override public synchronized GridCacheVersion versionedValue(CacheObject val,
GridCacheVersion curVer,
- GridCacheVersion newVer)
+ GridCacheVersion newVer,
+ @Nullable IgniteCacheExpiryPolicy loadExpiryPlc)
throws IgniteCheckedException, GridCacheEntryRemovedException {
checkObsolete();
@@ -3568,9 +3569,19 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
CacheObject old = rawGetOrUnmarshalUnlocked(false);
- long ttl = ttlExtras();
+ long ttl;
+ long expTime;
- long expTime = CU.toExpireTime(ttl);
+ if (loadExpiryPlc != null) {
+ IgniteBiTuple<Long, Long> initTtlAndExpireTime = initialTtlAndExpireTime(loadExpiryPlc);
+
+ ttl = initTtlAndExpireTime.get1();
+ expTime = initTtlAndExpireTime.get2();
+ }
+ else {
+ ttl = ttlExtras();
+ expTime = expireTimeExtras();
+ }
// Detach value before index update.
val = cctx.kernalContext().cacheObjects().prepareForCache(val, cctx);
http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
index 4566044..dd18d7a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
@@ -28,6 +28,7 @@ import java.util.ListIterator;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicReference;
+import javax.cache.expiry.ExpiryPolicy;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteLogger;
import org.apache.ignite.cluster.ClusterNode;
@@ -1059,10 +1060,28 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
try {
CacheObject val0 = cctx.toCacheObject(val);
+ long ttl = CU.TTL_ETERNAL;
+ long expireTime = CU.EXPIRE_TIME_ETERNAL;
+
+ ExpiryPolicy expiry = cctx.expiry();
+
+ if (expiry != null) {
+ ttl = CU.toTtl(expiry.getExpiryForCreation());
+
+ if (ttl == CU.TTL_ZERO)
+ expireTime = CU.expireTimeInPast();
+ else {
+ if (ttl == CU.TTL_NOT_CHANGED)
+ ttl = CU.TTL_ETERNAL;
+
+ expireTime = CU.toExpireTime(ttl);
+ }
+ }
+
entry0.initialValue(val0,
ver,
- 0,
- 0,
+ ttl,
+ expireTime,
false,
topVer,
GridDrType.DR_LOAD,
http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 0e60ff4..b291bd2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -2292,7 +2292,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
try {
GridCacheVersion ver = entry.version();
- entry.versionedValue(ctx.toCacheObject(v), null, ver);
+ entry.versionedValue(ctx.toCacheObject(v), null, ver, null);
}
catch (GridCacheEntryRemovedException e) {
assert false : "Entry should not get obsolete while holding lock [entry=" + entry +
http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 393fb1a..ba44655 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -475,7 +475,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
GridCacheEntryEx entry = cacheCtx.cache().entryEx(key, topVer);
try {
- GridCacheVersion setVer = entry.versionedValue(cacheVal, ver, null);
+ GridCacheVersion setVer = entry.versionedValue(cacheVal, ver, null, null);
boolean set = setVer != null;
http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
index 48621af..e3b4e9b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
@@ -667,7 +667,8 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
/** @inheritDoc */
@Override public GridCacheVersion versionedValue(CacheObject val,
GridCacheVersion curVer,
- GridCacheVersion newVer) {
+ GridCacheVersion newVer,
+ IgniteCacheExpiryPolicy loadExpiryPlc) {
assert false;
return null;
http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyWithStoreAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyWithStoreAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyWithStoreAbstractTest.java
index 78c59ac..58e6b02 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyWithStoreAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyWithStoreAbstractTest.java
@@ -17,6 +17,8 @@
package org.apache.ignite.internal.processors.cache.expiry;
+import java.util.ArrayList;
+import java.util.List;
import java.util.concurrent.TimeUnit;
import javax.cache.configuration.Factory;
import javax.cache.expiry.Duration;
@@ -174,6 +176,44 @@ public abstract class IgniteCacheExpiryPolicyWithStoreAbstractTest extends Ignit
}
/**
+ * @throws Exception If failed.
+ */
+ public void testGetReadThrough() throws Exception {
+ IgniteCache<Integer, Integer> cache = jcache(0);
+
+ List<Integer> keys = new ArrayList<>();
+
+ keys.add(primaryKeys(cache, 1, 100_000).get(0));
+ // TODO https://issues.apache.org/jira/browse/IGNITE-3699
+ // TODO: test 'get' inside transactions, 'get' for cache.withAsyncPolicy.
+ //keys.add(backupKeys(cache, 1, 100_000).get(0));
+ //keys.add(nearKeys(cache, 1, 100_000).get(0));
+
+ for (Integer key : keys)
+ storeMap.put(key, 100);
+
+ try {
+ for (Integer key : keys) {
+ Integer res = cache.get(key);
+
+ assertEquals((Integer)100, res);
+
+ checkTtl(key, 500, true);
+
+ assertEquals((Integer)100, res);
+ }
+
+ U.sleep(600);
+
+ for (Integer key : keys)
+ checkExpired(key);
+ }
+ finally {
+ cache.removeAll();
+ }
+ }
+
+ /**
* @param key Key.
*/
private void checkExpired(Integer key) {
[06/25] ignite git commit: IGNITE-3770 GridLogThrottle.warn ignores
the exception
Posted by sb...@apache.org.
IGNITE-3770 GridLogThrottle.warn ignores the exception
Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/bbaa79af
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/bbaa79af
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/bbaa79af
Branch: refs/heads/ignite-4371
Commit: bbaa79af8ef526b5d2684db0e3d71d60a8f1ebe7
Parents: ac8602d
Author: agura <ag...@apache.org>
Authored: Wed Dec 7 19:36:11 2016 +0300
Committer: agura <ag...@apache.org>
Committed: Wed Dec 7 19:36:11 2016 +0300
----------------------------------------------------------------------
.../affinity/fair/FairAffinityFunction.java | 2 +-
.../rendezvous/RendezvousAffinityFunction.java | 2 +-
.../apache/ignite/internal/IgniteKernal.java | 2 +-
.../GridDeploymentPerVersionStore.java | 2 +-
.../discovery/GridDiscoveryManager.java | 2 +-
.../eventstorage/GridEventStorageManager.java | 2 +-
.../cache/GridCacheDeploymentManager.java | 4 +-
.../processors/cache/GridCacheEventManager.java | 10 ++---
.../store/GridCacheStoreManagerAdapter.java | 2 +-
.../cache/store/GridCacheWriteBehindStore.java | 2 +-
.../clock/GridClockSyncProcessor.java | 2 +-
.../igfs/IgfsFragmentizerManager.java | 29 +++++++-------
.../internal/processors/igfs/IgfsImpl.java | 35 ++++++++---------
.../OsDiscoveryNodeValidationProcessor.java | 2 +-
.../processors/task/GridTaskWorker.java | 2 +-
.../ignite/internal/util/GridLogThrottle.java | 35 ++++++++---------
.../ignite/internal/util/IgniteUtils.java | 6 +--
.../ipc/shmem/IpcSharedMemoryNativeLoader.java | 2 +-
.../shmem/IpcSharedMemoryServerEndpoint.java | 4 +-
.../nio/GridConnectionBytesVerifyFilter.java | 2 +-
.../internal/util/nio/GridNioCodecFilter.java | 2 +-
.../internal/util/nio/GridNioFilterChain.java | 2 +-
.../ignite/internal/util/nio/GridNioServer.java | 2 +-
.../util/nio/GridSelectorNioSessionImpl.java | 2 +-
.../communication/tcp/TcpCommunicationSpi.java | 12 +++---
.../ignite/spi/discovery/tcp/ClientImpl.java | 7 ++--
.../ignite/spi/discovery/tcp/ServerImpl.java | 41 +++++++++-----------
.../spi/discovery/tcp/TcpDiscoveryImpl.java | 2 +-
.../spi/discovery/tcp/TcpDiscoverySpi.java | 8 ++--
.../TcpDiscoveryMulticastIpFinder.java | 4 +-
.../ignite/testframework/GridTestUtils.java | 2 +-
.../junits/common/GridCommonAbstractTest.java | 6 +--
.../apache/ignite/util/GridLogThrottleTest.java | 27 +++++--------
.../HadoopExternalCommunication.java | 6 +--
.../processors/query/h2/IgniteH2Indexing.java | 2 +-
.../cache/IgniteCacheOffheapEvictQueryTest.java | 2 +-
.../scanners/http/UriDeploymentHttpScanner.java | 8 ++--
37 files changed, 133 insertions(+), 151 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java
index 105efab..cffcf10 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java
@@ -331,7 +331,7 @@ public class FairAffinityFunction implements AffinityFunction {
balance(tier, pendingParts, fullMap, topSnapshot, true);
if (!exclNeighborsWarn) {
- LT.warn(log, null, "Affinity function excludeNeighbors property is ignored " +
+ LT.warn(log, "Affinity function excludeNeighbors property is ignored " +
"because topology has no enough nodes to assign backups.");
exclNeighborsWarn = true;
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
index 75e7c92..cbd0136 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
@@ -462,7 +462,7 @@ public class RendezvousAffinityFunction implements AffinityFunction, Externaliza
}
if (!exclNeighborsWarn) {
- LT.warn(log, null, "Affinity function excludeNeighbors property is ignored " +
+ LT.warn(log, "Affinity function excludeNeighbors property is ignored " +
"because topology has no enough nodes to assign backups.");
exclNeighborsWarn = true;
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index c536506..8fda72f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -1001,7 +1001,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
// at least one waiting request, then it is possible starvation.
if (exec.getPoolSize() == exec.getActiveCount() && completedCnt == lastCompletedCnt &&
!exec.getQueue().isEmpty())
- LT.warn(log, null, "Possible thread pool starvation detected (no task completed in last " +
+ LT.warn(log, "Possible thread pool starvation detected (no task completed in last " +
interval + "ms, is executorService pool size large enough?)");
lastCompletedCnt = completedCnt;
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentPerVersionStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentPerVersionStore.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentPerVersionStore.java
index 5e30bf6..0bf8328 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentPerVersionStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentPerVersionStore.java
@@ -317,7 +317,7 @@ public class GridDeploymentPerVersionStore extends GridDeploymentStoreAdapter {
if (ctx.localNodeId().equals(e.getKey())) {
// Warn only if mode is not CONTINUOUS.
if (meta.deploymentMode() != CONTINUOUS)
- LT.warn(log, null, "Local node is in participants (most probably, " +
+ LT.warn(log, "Local node is in participants (most probably, " +
"IgniteConfiguration.getPeerClassLoadingLocalClassPathExclude() " +
"is not used properly " +
"[locNodeId=" + ctx.localNodeId() + ", meta=" + meta + ']');
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index ddd4ee3..9aa4db1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -995,7 +995,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
break;
if (ctx.config().isWaitForSegmentOnStart()) {
- LT.warn(log, null, "Failed to check network segment (retrying every 2000 ms).");
+ LT.warn(log, "Failed to check network segment (retrying every 2000 ms).");
// Wait and check again.
U.sleep(2000);
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
index 5b451a1..607bb96 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
@@ -278,7 +278,7 @@ public class GridEventStorageManager extends GridManagerAdapter<EventStorageSpi>
int type = evt.type();
if (!isRecordable(type)) {
- LT.warn(log, null, "Trying to record event without checking if it is recordable: " +
+ LT.warn(log, "Trying to record event without checking if it is recordable: " +
U.gridEventName(type));
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
index 8e66233..ad4892b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
@@ -399,7 +399,7 @@ public class GridCacheDeploymentManager<K, V> extends GridCacheSharedManagerAdap
", daemon=" + daemon + ']');
if (!daemon) {
- LT.warn(log, null, "Ignoring deployment in PRIVATE or ISOLATED mode " +
+ LT.warn(log, "Ignoring deployment in PRIVATE or ISOLATED mode " +
"[sndId=" + sndId + ", ldrId=" + ldrId + ", userVer=" + userVer + ", mode=" + mode +
", participants=" + participants + ", daemon=" + daemon + ']');
}
@@ -408,7 +408,7 @@ public class GridCacheDeploymentManager<K, V> extends GridCacheSharedManagerAdap
}
if (mode != cctx.gridConfig().getDeploymentMode()) {
- LT.warn(log, null, "Local and remote deployment mode mismatch (please fix configuration and restart) " +
+ LT.warn(log, "Local and remote deployment mode mismatch (please fix configuration and restart) " +
"[locDepMode=" + cctx.gridConfig().getDeploymentMode() + ", rmtDepMode=" + mode + ", rmtNodeId=" +
sndId + ']');
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
index ec8b8cc..1c18738 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
@@ -251,7 +251,7 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
assert key != null || type == EVT_CACHE_STARTED || type == EVT_CACHE_STOPPED;
if (!cctx.events().isRecordable(type))
- LT.warn(log, null, "Added event without checking if event is recordable: " + U.gridEventName(type));
+ LT.warn(log, "Added event without checking if event is recordable: " + U.gridEventName(type));
// Events are not fired for internal entry.
if (key == null || !key.internal()) {
@@ -261,7 +261,7 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
evtNode = findNodeInHistory(evtNodeId);
if (evtNode == null)
- LT.warn(log, null, "Failed to find event node in grid topology history " +
+ LT.warn(log, "Failed to find event node in grid topology history " +
"(try to increase topology history size configuration property of configured " +
"discovery SPI): " + evtNodeId);
@@ -284,7 +284,7 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
log.debug("Failed to unmarshall cache object value for the event notification: " + e);
if (!forceKeepBinary)
- LT.warn(log, null, "Failed to unmarshall cache object value for the event notification " +
+ LT.warn(log, "Failed to unmarshall cache object value for the event notification " +
"(all further notifications will keep binary object format).");
forceKeepBinary = true;
@@ -351,7 +351,7 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
assert discoTs > 0;
if (!cctx.events().isRecordable(type))
- LT.warn(log, null, "Added event without checking if event is recordable: " + U.gridEventName(type));
+ LT.warn(log, "Added event without checking if event is recordable: " + U.gridEventName(type));
cctx.gridEvents().record(new CacheRebalancingEvent(cctx.name(), cctx.localNode(),
"Cache rebalancing event.", type, part, discoNode, discoType, discoTs));
@@ -364,7 +364,7 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
*/
public void addUnloadEvent(int part) {
if (!cctx.events().isRecordable(EVT_CACHE_REBALANCE_PART_UNLOADED))
- LT.warn(log, null, "Added event without checking if event is recordable: " +
+ LT.warn(log, "Added event without checking if event is recordable: " +
U.gridEventName(EVT_CACHE_REBALANCE_PART_UNLOADED));
cctx.gridEvents().record(new CacheRebalancingEvent(cctx.name(), cctx.localNode(),
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index cd0c50f..024375e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -547,7 +547,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
return true;
}
- LT.warn(log, null, "Calling Cache.loadCache() method will have no effect, " +
+ LT.warn(log, "Calling Cache.loadCache() method will have no effect, " +
"CacheConfiguration.getStore() is not defined for cache: " + cctx.namexx());
return false;
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
index 468945b..858d9a7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
@@ -705,7 +705,7 @@ public class GridCacheWriteBehindStore<K, V> implements CacheStore<K, V>, Lifecy
}
}
catch (Exception e) {
- LT.warn(log, e, "Unable to update underlying store: " + store);
+ LT.error(log, e, "Unable to update underlying store: " + store);
if (writeCache.sizex() > cacheCriticalSize || stopping.get()) {
for (Map.Entry<K, Entry<? extends K, ? extends V>> entry : vals.entrySet()) {
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
index b5c89cf..0764316 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
@@ -458,7 +458,7 @@ public class GridClockSyncProcessor extends GridProcessorAdapter {
srv.sendPacket(req, addr, port);
}
catch (IgniteCheckedException e) {
- LT.warn(log, e, "Failed to send time request to remote node [rmtNodeId=" + rmtNodeId +
+ LT.error(log, e, "Failed to send time request to remote node [rmtNodeId=" + rmtNodeId +
", addr=" + addr + ", port=" + port + ']');
}
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
index d64c64a..2e82f33 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
@@ -17,6 +17,19 @@
package org.apache.ignite.internal.processors.igfs;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteException;
import org.apache.ignite.cluster.ClusterNode;
@@ -41,20 +54,6 @@ import org.apache.ignite.lang.IgniteUuid;
import org.apache.ignite.thread.IgniteThread;
import org.jetbrains.annotations.Nullable;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.LinkedBlockingDeque;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
@@ -383,7 +382,7 @@ public class IgfsFragmentizerManager extends IgfsManager {
}
catch (IgniteCheckedException | IgniteException e) {
if (!X.hasCause(e, InterruptedException.class) && !X.hasCause(e, IgniteInterruptedCheckedException.class))
- LT.warn(log, e, "Failed to get fragmentizer file info (will retry).");
+ LT.error(log, e, "Failed to get fragmentizer file info (will retry).");
else {
if (log.isDebugEnabled())
log.debug("Got interrupted exception in fragmentizer coordinator (grid is stopping).");
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
index 1c985c0..ab4ee85 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
@@ -17,6 +17,21 @@
package org.apache.ignite.internal.processors.igfs;
+import java.io.OutputStream;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteException;
@@ -81,22 +96,6 @@ import org.apache.ignite.thread.IgniteThreadPoolExecutor;
import org.jetbrains.annotations.Nullable;
import org.jsr166.ConcurrentHashMap8;
-import java.io.OutputStream;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_DELETED;
import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_DELETED;
import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_OPENED_READ;
@@ -1308,7 +1307,7 @@ public final class IgfsImpl implements IgfsEx {
secondarySpaceSize = secondaryFs.usedSpaceSize();
}
catch (IgniteException e) {
- LT.warn(log, e, "Failed to get secondary file system consumed space size.");
+ LT.error(log, e, "Failed to get secondary file system consumed space size.");
secondarySpaceSize = -1;
}
@@ -1841,4 +1840,4 @@ public final class IgfsImpl implements IgfsEx {
}
}
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/processors/nodevalidation/OsDiscoveryNodeValidationProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/nodevalidation/OsDiscoveryNodeValidationProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/nodevalidation/OsDiscoveryNodeValidationProcessor.java
index a7e06e9..37e59bc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/nodevalidation/OsDiscoveryNodeValidationProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/nodevalidation/OsDiscoveryNodeValidationProcessor.java
@@ -58,7 +58,7 @@ public class OsDiscoveryNodeValidationProcessor extends GridProcessorAdapter imp
", rmtNodeAddrs=" + U.addressesAsString(node) +
", locNodeId=" + locNode.id() + ", rmtNodeId=" + node.id() + ']';
- LT.warn(log, null, errMsg);
+ LT.warn(log, errMsg);
// Always output in debug.
if (log.isDebugEnabled())
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
index 0be69d1..3478c70 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
@@ -626,7 +626,7 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
res.setOccupied(true);
if (resCache && jobRes.size() > ctx.discovery().size() && jobRes.size() % SPLIT_WARN_THRESHOLD == 0)
- LT.warn(log, null, "Number of jobs in task is too large for task: " + ses.getTaskName() +
+ LT.warn(log, "Number of jobs in task is too large for task: " + ses.getTaskName() +
". Consider reducing number of jobs or disabling job result cache with " +
"@ComputeTaskNoResultCache annotation.");
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java
index c8ba865..7f30dd7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java
@@ -93,13 +93,12 @@ public class GridLogThrottle {
* Logs warning if needed.
*
* @param log Logger.
- * @param e Error (optional).
* @param msg Message.
*/
- public static void warn(@Nullable IgniteLogger log, @Nullable Throwable e, String msg) {
+ public static void warn(@Nullable IgniteLogger log, String msg) {
assert !F.isEmpty(msg);
- log(log, e, msg, null, LogLevel.WARN, false, false);
+ log(log, null, msg, null, LogLevel.WARN, false, false);
}
/**
@@ -122,28 +121,26 @@ public class GridLogThrottle {
* Logs warning if needed.
*
* @param log Logger.
- * @param e Error (optional).
* @param msg Message.
- * @param quite Print warning anyway.
+ * @param quiet Print warning anyway.
*/
- public static void warn(@Nullable IgniteLogger log, @Nullable Throwable e, String msg, boolean quite) {
+ public static void warn(@Nullable IgniteLogger log, String msg, boolean quiet) {
assert !F.isEmpty(msg);
- log(log, e, msg, null, LogLevel.WARN, quite, false);
+ log(log, null, msg, null, LogLevel.WARN, quiet, false);
}
/**
* Logs warning if needed.
*
* @param log Logger.
- * @param e Error (optional).
* @param longMsg Long message (or just message).
- * @param shortMsg Short message for quite logging.
+ * @param shortMsg Short message for quiet logging.
*/
- public static void warn(@Nullable IgniteLogger log, @Nullable Throwable e, String longMsg, @Nullable String shortMsg) {
+ public static void warn(@Nullable IgniteLogger log, String longMsg, @Nullable String shortMsg) {
assert !F.isEmpty(longMsg);
- log(log, e, longMsg, shortMsg, LogLevel.WARN, false, false);
+ log(log, null, longMsg, shortMsg, LogLevel.WARN, false, false);
}
/**
@@ -151,12 +148,12 @@ public class GridLogThrottle {
*
* @param log Logger.
* @param msg Message.
- * @param quite Print info anyway.
+ * @param quiet Print info anyway.
*/
- public static void info(@Nullable IgniteLogger log, String msg, boolean quite) {
+ public static void info(@Nullable IgniteLogger log, String msg, boolean quiet) {
assert !F.isEmpty(msg);
- log(log, null, msg, null, LogLevel.INFO, quite, false);
+ log(log, null, msg, null, LogLevel.INFO, quiet, false);
}
/**
@@ -166,6 +163,8 @@ public class GridLogThrottle {
* @param msg Message.
*/
public static void info(@Nullable IgniteLogger log, String msg) {
+ assert !F.isEmpty(msg);
+
info(log, msg, false);
}
@@ -182,13 +181,13 @@ public class GridLogThrottle {
* @param log Logger.
* @param e Error (optional).
* @param longMsg Long message (or just message).
- * @param shortMsg Short message for quite logging.
+ * @param shortMsg Short message for quiet logging.
* @param level Level where messages should appear.
* @param byMsg Errors group by message, not by tuple(error, msg).
*/
@SuppressWarnings({"RedundantTypeArguments"})
- private static void log(@Nullable IgniteLogger log, @Nullable Throwable e, String longMsg, @Nullable String shortMsg,
- LogLevel level, boolean quiet, boolean byMsg) {
+ private static void log(@Nullable IgniteLogger log, @Nullable Throwable e, String longMsg,
+ @Nullable String shortMsg, LogLevel level, boolean quiet, boolean byMsg) {
assert !F.isEmpty(longMsg);
IgniteBiTuple<Class<? extends Throwable>, String> tup =
@@ -283,4 +282,4 @@ public class GridLogThrottle {
*/
public abstract void doLog(IgniteLogger log, String longMsg, String shortMsg, Throwable e, boolean quiet);
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
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 914b3ec..3fa3f7b 100644
--- 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
@@ -4105,7 +4105,7 @@ public abstract class IgniteUtils {
}
/**
- * Logs warning message in both verbose and quite modes.
+ * Logs warning message in both verbose and quiet modes.
*
* @param log Logger to use.
* @param msg Message to log.
@@ -4115,7 +4115,7 @@ public abstract class IgniteUtils {
}
/**
- * Logs warning message in both verbose and quite modes.
+ * Logs warning message in both verbose and quiet modes.
*
* @param log Logger to use.
* @param shortMsg Short message.
@@ -4285,7 +4285,7 @@ public abstract class IgniteUtils {
}
/**
- * Prints out the message in quite and info modes.
+ * Prints out the message in quiet and info modes.
*
* @param log Logger.
* @param msg Message to print.
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java
index 2771d28..02c4de5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java
@@ -150,7 +150,7 @@ public class IpcSharedMemoryNativeLoader {
try {
if (log != null)
- LT.warn(log, null, "Failed to load 'igniteshmem' library from classpath. Will try to load it from IGNITE_HOME.");
+ LT.warn(log, "Failed to load 'igniteshmem' library from classpath. Will try to load it from IGNITE_HOME.");
String igniteHome = X.resolveIgniteHome();
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
index 94c3820..178e608 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
@@ -304,13 +304,13 @@ public class IpcSharedMemoryServerEndpoint implements IpcServerEndpoint {
String msg = "Failed to process incoming connection (most probably, shared memory " +
"rest endpoint has been configured by mistake).";
- LT.warn(log, null, msg);
+ LT.warn(log, msg);
sendErrorResponse(out, e);
}
catch (IpcOutOfSystemResourcesException e) {
if (!omitOutOfResourcesWarn)
- LT.warn(log, null, OUT_OF_RESOURCES_MSG);
+ LT.warn(log, OUT_OF_RESOURCES_MSG);
sendErrorResponse(out, e);
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
index 13d7ca7..213fd8d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
@@ -115,7 +115,7 @@ public class GridConnectionBytesVerifyFilter extends GridNioFilterAdapter {
else {
ses.close();
- LT.warn(log, null, "Unknown connection detected (is some other software connecting to this " +
+ LT.warn(log, "Unknown connection detected (is some other software connecting to this " +
"Ignite port?) [rmtAddr=" + ses.remoteAddress() + ", locAddr=" + ses.localAddress() + ']');
}
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
index a2f543d..7083ccf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
@@ -110,7 +110,7 @@ public class GridNioCodecFilter extends GridNioFilterAdapter {
if (directMode)
return;
- LT.warn(log, null, "Parser returned null but there are still unread data in input buffer (bug in " +
+ LT.warn(log, "Parser returned null but there are still unread data in input buffer (bug in " +
"parser code?) [parser=" + parser + ", ses=" + ses + ']');
input.position(input.limit());
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
index 8a43e29..a3a74e3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
@@ -158,7 +158,7 @@ public class GridNioFilterChain<T> extends GridNioFilterAdapter {
head.onExceptionCaught(ses, e);
}
catch (Exception ex) {
- LT.warn(log, ex, "Failed to forward GridNioException to filter chain [ses=" + ses + ", e=" + e + ']');
+ LT.error(log, ex, "Failed to forward GridNioException to filter chain [ses=" + ses + ", e=" + e + ']');
}
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index 24b8fad..c8e2e0b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -768,7 +768,7 @@ public class GridNioServer<T> {
filterChain.onMessageReceived(ses, readBuf);
if (readBuf.remaining() > 0) {
- LT.warn(log, null, "Read buffer contains data after filter chain processing (will discard " +
+ LT.warn(log, "Read buffer contains data after filter chain processing (will discard " +
"remaining bytes) [ses=" + ses + ", remainingCnt=" + readBuf.remaining() + ']');
readBuf.clear();
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
index 0ba6af2..63c9845 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
@@ -227,7 +227,7 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
if (recovery != null) {
if (!recovery.add(last)) {
- LT.warn(log, null, "Unacknowledged messages queue size overflow, will attempt to reconnect " +
+ LT.warn(log, "Unacknowledged messages queue size overflow, will attempt to reconnect " +
"[remoteAddr=" + remoteAddress() +
", queueLimit=" + recovery.queueLimit() + ']');
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 767490f..1fe437c 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -331,7 +331,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
private final GridNioServerListener<Message> srvLsnr =
new GridNioServerListenerAdapter<Message>() {
@Override public void onSessionWriteTimeout(GridNioSession ses) {
- LT.warn(log, null, "Communication SPI Session write timed out (consider increasing " +
+ LT.warn(log, "Communication SPI Session write timed out (consider increasing " +
"'socketWriteTimeout' " + "configuration property) [remoteAddr=" + ses.remoteAddress() +
", writeTimeout=" + sockWriteTimeout + ']');
@@ -2146,9 +2146,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
catch (IgniteCheckedException e) {
if (e.hasCause(IpcOutOfSystemResourcesException.class))
// Has cause or is itself the IpcOutOfSystemResourcesException.
- LT.warn(log, null, OUT_OF_RESOURCES_TCP_MSG);
+ LT.warn(log, OUT_OF_RESOURCES_TCP_MSG);
else if (getSpiContext().node(node.id()) != null)
- LT.warn(log, null, e.getMessage());
+ LT.warn(log, e.getMessage());
else if (log.isDebugEnabled())
log.debug("Failed to establish shared memory connection with local node (node has left): " +
node.id());
@@ -2510,11 +2510,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
boolean failureDetThrReached = timeoutHelper.checkFailureTimeoutReached(e);
if (failureDetThrReached)
- LT.warn(log, null, "Connect timed out (consider increasing 'failureDetectionTimeout' " +
+ LT.warn(log, "Connect timed out (consider increasing 'failureDetectionTimeout' " +
"configuration property) [addr=" + addr + ", failureDetectionTimeout=" +
failureDetectionTimeout() + ']');
else if (X.hasCause(e, SocketTimeoutException.class))
- LT.warn(log, null, "Connect timed out (consider increasing 'connTimeout' " +
+ LT.warn(log, "Connect timed out (consider increasing 'connTimeout' " +
"configuration property) [addr=" + addr + ", connTimeout=" + connTimeout + ']');
if (errs == null)
@@ -2545,7 +2545,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
assert errs != null;
if (X.hasCause(errs, ConnectException.class))
- LT.warn(log, null, "Failed to connect to a remote node " +
+ LT.warn(log, "Failed to connect to a remote node " +
"(make sure that destination node is alive and " +
"operating system firewall is disabled on local and remote hosts) " +
"[addrs=" + addrs + ']');
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
index 7b8c029..8928f28 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -64,7 +64,6 @@ import org.apache.ignite.internal.util.typedef.internal.LT;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.lang.IgniteInClosure;
import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.marshaller.MarshallerUtils;
import org.apache.ignite.spi.IgniteSpiContext;
import org.apache.ignite.spi.IgniteSpiException;
import org.apache.ignite.spi.IgniteSpiOperationTimeoutHelper;
@@ -483,7 +482,7 @@ class ClientImpl extends TcpDiscoveryImpl {
if (timeout > 0 && (U.currentTimeMillis() - startTime) > timeout)
return null;
- LT.warn(log, null, "IP finder returned empty addresses list. " +
+ LT.warn(log, "IP finder returned empty addresses list. " +
"Please check IP finder configuration" +
(spi.ipFinder instanceof TcpDiscoveryMulticastIpFinder ?
" and make sure multicast works on your network. " : ". ") +
@@ -553,7 +552,7 @@ class ClientImpl extends TcpDiscoveryImpl {
if (timeout > 0 && (U.currentTimeMillis() - startTime) > timeout)
return null;
- LT.warn(log, null, "Failed to connect to any address from IP finder (will retry to join topology " +
+ LT.warn(log, "Failed to connect to any address from IP finder (will retry to join topology " +
"every 2 secs): " + toOrderedList(addrs0), true);
Thread.sleep(2000);
@@ -917,7 +916,7 @@ class ClientImpl extends TcpDiscoveryImpl {
ClassNotFoundException clsNotFoundEx = X.cause(e, ClassNotFoundException.class);
if (clsNotFoundEx != null)
- LT.warn(log, null, "Failed to read message due to ClassNotFoundException " +
+ LT.warn(log, "Failed to read message due to ClassNotFoundException " +
"(make sure same versions of all classes are available on all nodes) " +
"[rmtNodeId=" + rmtNodeId + ", err=" + clsNotFoundEx.getMessage() + ']');
else
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index a660ec8..204b685 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -54,7 +54,6 @@ import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.LinkedBlockingDeque;
import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import javax.net.ssl.SSLException;
@@ -529,7 +528,7 @@ class ServerImpl extends TcpDiscoveryImpl {
boolean res = pingNode(node);
if (!res && !node.isClient() && nodeAlive(nodeId)) {
- LT.warn(log, null, "Failed to ping node (status check will be initiated): " + nodeId);
+ LT.warn(log, "Failed to ping node (status check will be initiated): " + nodeId);
msgWorker.addMessage(new TcpDiscoveryStatusCheckMessage(locNode, node.id()));
}
@@ -910,7 +909,7 @@ class ServerImpl extends TcpDiscoveryImpl {
U.addressesAsString(msg.addresses(), msg.hostNames()) + ']');
}
else
- LT.warn(log, null, "Node has not been connected to topology and will repeat join process. " +
+ LT.warn(log, "Node has not been connected to topology and will repeat join process. " +
"Check remote nodes logs for possible error messages. " +
"Note that large topology may require significant time to start. " +
"Increase 'TcpDiscoverySpi.networkTimeout' configuration property " +
@@ -1059,7 +1058,7 @@ class ServerImpl extends TcpDiscoveryImpl {
}
if (e != null && X.hasCause(e, ConnectException.class)) {
- LT.warn(log, null, "Failed to connect to any address from IP finder " +
+ LT.warn(log, "Failed to connect to any address from IP finder " +
"(make sure IP finder addresses are correct and firewalls are disabled on all host machines): " +
toOrderedList(addrs), true);
}
@@ -3148,7 +3147,7 @@ class ServerImpl extends TcpDiscoveryImpl {
}
}
- LT.warn(log, null, "Local node has detected failed nodes and started cluster-wide procedure. " +
+ LT.warn(log, "Local node has detected failed nodes and started cluster-wide procedure. " +
"To speed up failure detection please see 'Failure Detection' section under javadoc" +
" for 'TcpDiscoverySpi'");
}
@@ -3233,7 +3232,7 @@ class ServerImpl extends TcpDiscoveryImpl {
"[locNodeAddrs=" + U.addressesAsString(locNode) +
", rmtNodeAddrs=" + U.addressesAsString(node) + ']';
- LT.warn(log, null, errMsg);
+ LT.warn(log, errMsg);
// Always output in debug.
if (log.isDebugEnabled())
@@ -3286,7 +3285,7 @@ class ServerImpl extends TcpDiscoveryImpl {
}
// Output warning.
- LT.warn(log, null, "Ignoring join request from node (duplicate ID) [node=" + node +
+ LT.warn(log, "Ignoring join request from node (duplicate ID) [node=" + node +
", existingNode=" + existingNode + ']');
// Ignore join request.
@@ -3341,8 +3340,7 @@ class ServerImpl extends TcpDiscoveryImpl {
if (subj == null) {
// Node has not pass authentication.
- LT.warn(log, null,
- "Authentication failed [nodeId=" + node.id() +
+ LT.warn(log, "Authentication failed [nodeId=" + node.id() +
", addrs=" + U.addressesAsString(node) + ']',
"Authentication failed [nodeId=" + U.id8(node.id()) + ", addrs=" +
U.addressesAsString(node) + ']');
@@ -3371,8 +3369,7 @@ class ServerImpl extends TcpDiscoveryImpl {
else {
if (!(subj instanceof Serializable)) {
// Node has not pass authentication.
- LT.warn(log, null,
- "Authentication subject is not Serializable [nodeId=" + node.id() +
+ LT.warn(log, "Authentication subject is not Serializable [nodeId=" + node.id() +
", addrs=" + U.addressesAsString(node) + ']',
"Authentication subject is not Serializable [nodeId=" + U.id8(node.id()) +
", addrs=" +
@@ -3442,7 +3439,7 @@ class ServerImpl extends TcpDiscoveryImpl {
return;
}
- LT.warn(log, null, err.message());
+ LT.warn(log, err.message());
// Always output in debug.
if (log.isDebugEnabled())
@@ -3483,7 +3480,7 @@ class ServerImpl extends TcpDiscoveryImpl {
", rmtNodeAddrs=" + U.addressesAsString(node) +
", locNodeId=" + locNode.id() + ", rmtNodeId=" + msg.creatorNodeId() + ']';
- LT.warn(log, null, errMsg);
+ LT.warn(log, errMsg);
// Always output in debug.
if (log.isDebugEnabled())
@@ -3771,7 +3768,7 @@ class ServerImpl extends TcpDiscoveryImpl {
* @param sndMsg Message to send.
*/
private void nodeCheckError(TcpDiscoveryNode node, String errMsg, String sndMsg) {
- LT.warn(log, null, errMsg);
+ LT.warn(log, errMsg);
// Always output in debug.
if (log.isDebugEnabled())
@@ -4056,8 +4053,7 @@ class ServerImpl extends TcpDiscoveryImpl {
if (!permissionsEqual(coordSubj.subject().permissions(), subj.subject().permissions())) {
// Node has not pass authentication.
- LT.warn(log, null,
- "Authentication failed [nodeId=" + node.id() +
+ LT.warn(log, "Authentication failed [nodeId=" + node.id() +
", addrs=" + U.addressesAsString(node) + ']',
"Authentication failed [nodeId=" + U.id8(node.id()) + ", addrs=" +
U.addressesAsString(node) + ']');
@@ -4148,7 +4144,6 @@ class ServerImpl extends TcpDiscoveryImpl {
rmCrd.subject().permissions())) {
// Node has not pass authentication.
LT.warn(log,
- null,
"Failed to authenticate local node " +
"(local authentication result is different from rest of topology) " +
"[nodeId=" + node.id() + ", addrs=" + U.addressesAsString(node) + ']',
@@ -5593,7 +5588,7 @@ class ServerImpl extends TcpDiscoveryImpl {
"[rmtAddr=" + sock.getRemoteSocketAddress() +
", locAddr=" + sock.getLocalSocketAddress() + ']');
- LT.warn(log, null, "Failed to read magic header (too few bytes received) [rmtAddr=" +
+ LT.warn(log, "Failed to read magic header (too few bytes received) [rmtAddr=" +
sock.getRemoteSocketAddress() + ", locAddr=" + sock.getLocalSocketAddress() + ']');
return;
@@ -5609,7 +5604,7 @@ class ServerImpl extends TcpDiscoveryImpl {
"[rmtAddr=" + sock.getRemoteSocketAddress() +
", locAddr=" + sock.getLocalSocketAddress() + ']');
- LT.warn(log, null, "Unknown connection detected (is some other software connecting to " +
+ LT.warn(log, "Unknown connection detected (is some other software connecting to " +
"this Ignite port?" +
(!spi.isSslEnabled() ? " missing SSL configuration on remote node?" : "" ) +
") [rmtAddr=" + sock.getInetAddress() + ']', true);
@@ -5729,7 +5724,7 @@ class ServerImpl extends TcpDiscoveryImpl {
U.error(log, "Caught exception on handshake [err=" + e +", sock=" + sock + ']', e);
if (X.hasCause(e, SSLException.class) && spi.isSslEnabled() && !spi.isNodeStopping0())
- LT.warn(log, null, "Failed to initialize connection " +
+ LT.warn(log, "Failed to initialize connection " +
"(missing SSL configuration on remote node?) " +
"[rmtAddr=" + sock.getInetAddress() + ']', true);
else if ((X.hasCause(e, ObjectStreamException.class) || !sock.isClosed())
@@ -5758,12 +5753,12 @@ class ServerImpl extends TcpDiscoveryImpl {
onException("Caught exception on handshake [err=" + e +", sock=" + sock + ']', e);
if (e.hasCause(SocketTimeoutException.class))
- LT.warn(log, null, "Socket operation timed out on handshake " +
+ LT.warn(log, "Socket operation timed out on handshake " +
"(consider increasing 'networkTimeout' configuration property) " +
"[netTimeout=" + spi.netTimeout + ']');
else if (e.hasCause(ClassNotFoundException.class))
- LT.warn(log, null, "Failed to read message due to ClassNotFoundException " +
+ LT.warn(log, "Failed to read message due to ClassNotFoundException " +
"(make sure same versions of all classes are available on all nodes) " +
"[rmtAddr=" + sock.getRemoteSocketAddress() +
", err=" + X.cause(e, ClassNotFoundException.class).getMessage() + ']');
@@ -5995,7 +5990,7 @@ class ServerImpl extends TcpDiscoveryImpl {
return;
if (e.hasCause(ClassNotFoundException.class))
- LT.warn(log, null, "Failed to read message due to ClassNotFoundException " +
+ LT.warn(log, "Failed to read message due to ClassNotFoundException " +
"(make sure same versions of all classes are available on all nodes) " +
"[rmtNodeId=" + nodeId +
", err=" + X.cause(e, ClassNotFoundException.class).getMessage() + ']');
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
index 0816cbc..f199c20 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
@@ -329,7 +329,7 @@ abstract class TcpDiscoveryImpl {
*/
protected boolean checkAckTimeout(long ackTimeout) {
if (ackTimeout > spi.getMaxAckTimeout()) {
- LT.warn(log, null, "Acknowledgement timeout is greater than maximum acknowledgement timeout " +
+ LT.warn(log, "Acknowledgement timeout is greater than maximum acknowledgement timeout " +
"(consider increasing 'maxAckTimeout' configuration property) " +
"[ackTimeout=" + ackTimeout + ", maxAckTimeout=" + spi.getMaxAckTimeout() + ']');
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index a8704e7..45933e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -1471,7 +1471,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
}
catch (IOException | IgniteCheckedException e) {
if (X.hasCause(e, SocketTimeoutException.class))
- LT.warn(log, null, "Timed out waiting for message to be read (most probably, the reason is " +
+ LT.warn(log, "Timed out waiting for message to be read (most probably, the reason is " +
"in long GC pauses on remote node) [curTimeout=" + timeout + ']');
throw e;
@@ -1511,7 +1511,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
return res;
}
catch (SocketTimeoutException e) {
- LT.warn(log, null, "Timed out waiting for message delivery receipt (most probably, the reason is " +
+ LT.warn(log, "Timed out waiting for message delivery receipt (most probably, the reason is " +
"in long GC pauses on remote node; consider tuning GC and increasing 'ackTimeout' " +
"configuration property). Will retry to send message with increased timeout. " +
"Current timeout: " + timeout + '.');
@@ -1575,7 +1575,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
res.add(resolved);
}
catch (UnknownHostException ignored) {
- LT.warn(log, null, "Failed to resolve address from IP finder (host is unknown): " + addr);
+ LT.warn(log, "Failed to resolve address from IP finder (host is unknown): " + addr);
// Add address in any case.
res.add(addr);
@@ -2045,7 +2045,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
// Close socket - timeout occurred.
U.closeQuiet(sock);
- LT.warn(log, null, "Socket write has timed out (consider increasing " +
+ LT.warn(log, "Socket write has timed out (consider increasing " +
(failureDetectionTimeoutEnabled() ?
"'IgniteConfiguration.failureDetectionTimeout' configuration property) [" +
"failureDetectionTimeout=" + failureDetectionTimeout() + ']' :
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
index e96abe9..8fe8a65 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
@@ -598,7 +598,7 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
addrRes = new AddressResponse(data);
}
catch (IgniteCheckedException e) {
- LT.warn(log, e, "Failed to deserialize multicast response.");
+ LT.error(log, e, "Failed to deserialize multicast response.");
continue;
}
@@ -876,7 +876,7 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
}
catch (IOException e) {
if (!isInterrupted()) {
- LT.warn(log, e, "Failed to send/receive address message (will try to reconnect).");
+ LT.error(log, e, "Failed to send/receive address message (will try to reconnect).");
synchronized (this) {
U.close(sock);
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
index b3ce46b..0ae6575 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
@@ -1061,7 +1061,7 @@ public final class GridTestUtils {
Collection<ClusterNode> nodes = top.nodes(p, AffinityTopologyVersion.NONE);
if (nodes.size() > backups + 1) {
- LT.warn(log, null, "Partition map was not updated yet (will wait) [grid=" + g.name() +
+ LT.warn(log, "Partition map was not updated yet (will wait) [grid=" + g.name() +
", p=" + p + ", nodes=" + F.nodeIds(nodes) + ']');
wait = true;
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index d5f6b1c..90fabd9 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -507,7 +507,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
if (affNodes.size() != owners.size() || !affNodes.containsAll(owners) ||
(waitEvicts && loc != null && loc.state() != GridDhtPartitionState.OWNING)) {
- LT.warn(log(), null, "Waiting for topology map update [" +
+ LT.warn(log(), "Waiting for topology map update [" +
"grid=" + g.name() +
", cache=" + cfg.getName() +
", cacheId=" + dht.context().cacheId() +
@@ -524,7 +524,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
match = true;
}
else {
- LT.warn(log(), null, "Waiting for topology map update [" +
+ LT.warn(log(), "Waiting for topology map update [" +
"grid=" + g.name() +
", cache=" + cfg.getName() +
", cacheId=" + dht.context().cacheId() +
@@ -590,7 +590,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
}
if (entry.getValue() != GridDhtPartitionState.OWNING) {
- LT.warn(log(), null,
+ LT.warn(log(),
"Waiting for correct partition state, should be OWNING [state=" +
entry.getValue() + "]");
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/test/java/org/apache/ignite/util/GridLogThrottleTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridLogThrottleTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridLogThrottleTest.java
index d9540a8..9eac0cc 100644
--- a/modules/core/src/test/java/org/apache/ignite/util/GridLogThrottleTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/util/GridLogThrottleTest.java
@@ -53,26 +53,20 @@ public class GridLogThrottleTest extends GridCommonAbstractTest {
// LOGGED.
LT.error(log, new RuntimeException("Test exception 2."), "Test");
- // OMITTED.
- LT.warn(log, new RuntimeException("Test exception 1."), "Test");
-
- // OMITTED.
- LT.warn(log, new RuntimeException("Test exception 2."), "Test1");
-
- // OMITTED.
- LT.warn(log, new RuntimeException("Test exception 2."), "Test3");
-
// LOGGED.
LT.error(log, null, "Test - without throwable.");
// OMITTED.
LT.error(log, null, "Test - without throwable.");
+ // OMITTED.
+ LT.warn(log, "Test - without throwable.");
+
// LOGGED.
- LT.warn(log, null, "Test - without throwable1.");
+ LT.warn(log, "Test - without throwable1.");
// OMITTED.
- LT.warn(log, null, "Test - without throwable1.");
+ LT.warn(log, "Test - without throwable1.");
Thread.sleep(LT.throttleTimeout());
@@ -90,14 +84,11 @@ public class GridLogThrottleTest extends GridCommonAbstractTest {
// LOGGED.
LT.error(log, new RuntimeException("Test exception 2."), "Test");
- // OMITTED.
- LT.warn(log, new RuntimeException("Test exception 1."), "Test");
-
- // OMITTED.
- LT.warn(log, new RuntimeException("Test exception 2."), "Test1");
+ // LOGGED.
+ LT.warn(log, "Test - without throwable.");
// OMITTED.
- LT.warn(log, new RuntimeException("Test exception 2."), "Test3");
+ LT.warn(log, "Test - without throwable.");
Thread.sleep(LT.throttleTimeout());
@@ -121,4 +112,4 @@ public class GridLogThrottleTest extends GridCommonAbstractTest {
//OMMITED.
LT.info(log(), "Test info message.");
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
index 6d903d8..bc047e7 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
@@ -856,7 +856,7 @@ public class HadoopExternalCommunication {
catch (IgniteCheckedException e) {
if (e.hasCause(IpcOutOfSystemResourcesException.class))
// Has cause or is itself the IpcOutOfSystemResourcesException.
- LT.warn(log, null, OUT_OF_RESOURCES_TCP_MSG);
+ LT.warn(log, OUT_OF_RESOURCES_TCP_MSG);
else if (log.isDebugEnabled())
log.debug("Failed to establish shared memory connection with local hadoop process: " +
desc);
@@ -1059,7 +1059,7 @@ public class HadoopExternalCommunication {
", err=" + e + ']');
if (X.hasCause(e, SocketTimeoutException.class))
- LT.warn(log, null, "Connect timed out (consider increasing 'connTimeout' " +
+ LT.warn(log, "Connect timed out (consider increasing 'connTimeout' " +
"configuration property) [addr=" + addr + ", port=" + port + ']');
if (errs == null)
@@ -1084,7 +1084,7 @@ public class HadoopExternalCommunication {
assert errs != null;
if (X.hasCause(errs, ConnectException.class))
- LT.warn(log, null, "Failed to connect to a remote Hadoop process (is process still running?). " +
+ LT.warn(log, "Failed to connect to a remote Hadoop process (is process still running?). " +
"Make sure operating system firewall is disabled on local and remote host) " +
"[addrs=" + addr + ", port=" + port + ']');
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index c3f701a..6da8758 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -996,7 +996,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
String longMsg = "Query execution is too long [time=" + time + " ms, sql='" + sql + '\'' +
", plan=" + U.nl() + plan.getString(1) + U.nl() + ", parameters=" + params + "]";
- LT.warn(log, null, longMsg, msg);
+ LT.warn(log, longMsg, msg);
}
return rs;
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
index c24fed4..bddef66 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
@@ -181,7 +181,7 @@ public class IgniteCacheOffheapEvictQueryTest extends GridCommonAbstractTest {
}
}
- LT.warn(log, null, e.getMessage());
+ LT.warn(log, e.getMessage());
return;
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/scanners/http/UriDeploymentHttpScanner.java
----------------------------------------------------------------------
diff --git a/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/scanners/http/UriDeploymentHttpScanner.java b/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/scanners/http/UriDeploymentHttpScanner.java
index 48bfd7f..bb7260d 100644
--- a/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/scanners/http/UriDeploymentHttpScanner.java
+++ b/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/scanners/http/UriDeploymentHttpScanner.java
@@ -343,11 +343,11 @@ public class UriDeploymentHttpScanner implements UriDeploymentScanner {
catch (IOException e) {
if (!scanCtx.isCancelled()) {
if (X.hasCause(e, ConnectException.class)) {
- LT.warn(scanCtx.getLogger(), e, "Failed to connect to HTTP server " +
+ LT.error(scanCtx.getLogger(), e, "Failed to connect to HTTP server " +
"(connection refused): " + U.hidePassword(url));
}
else if (X.hasCause(e, UnknownHostException.class)) {
- LT.warn(scanCtx.getLogger(), e, "Failed to connect to HTTP server " +
+ LT.error(scanCtx.getLogger(), e, "Failed to connect to HTTP server " +
"(host is unknown): " + U.hidePassword(url));
}
else
@@ -404,11 +404,11 @@ public class UriDeploymentHttpScanner implements UriDeploymentScanner {
catch (IOException e) {
if (!scanCtx.isCancelled()) {
if (X.hasCause(e, ConnectException.class)) {
- LT.warn(scanCtx.getLogger(), e, "Failed to connect to HTTP server (connection refused): " +
+ LT.error(scanCtx.getLogger(), e, "Failed to connect to HTTP server (connection refused): " +
U.hidePassword(url.toString()));
}
else if (X.hasCause(e, UnknownHostException.class)) {
- LT.warn(scanCtx.getLogger(), e, "Failed to connect to HTTP server (host is unknown): " +
+ LT.error(scanCtx.getLogger(), e, "Failed to connect to HTTP server (host is unknown): " +
U.hidePassword(url.toString()));
}
else
[05/25] ignite git commit: ignite-3685 - fixed
Posted by sb...@apache.org.
ignite-3685 - fixed
Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/ac8602db
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/ac8602db
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/ac8602db
Branch: refs/heads/ignite-4371
Commit: ac8602dbdf2bbf5b16a611eaf6d520a0a7b0010b
Parents: 0d4a1b7
Author: Sergi Vladykin <se...@gmail.com>
Authored: Mon Aug 15 16:46:54 2016 +0300
Committer: Sergi <se...@gmail.com>
Committed: Wed Dec 7 15:08:12 2016 +0300
----------------------------------------------------------------------
.../internal/processors/query/h2/opt/GridH2Row.java | 2 +-
.../IgniteCacheAbstractFieldsQuerySelfTest.java | 2 +-
.../local/IgniteCacheLocalFieldsQuerySelfTest.java | 16 ++++++++++++++++
3 files changed, 18 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/ac8602db/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
index 9486a2e..8e7b161 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
@@ -98,7 +98,7 @@ public abstract class GridH2Row extends Row implements GridSearchRowPointer {
/** {@inheritDoc} */
@Override public void setKey(long key) {
- throw new UnsupportedOperationException();
+ // No-op, may be set in H2 INFORMATION_SCHEMA.
}
/** {@inheritDoc} */
http://git-wip-us.apache.org/repos/asf/ignite/blob/ac8602db/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
index 926d294..d5f02eb 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
@@ -71,7 +71,7 @@ public abstract class IgniteCacheAbstractFieldsQuerySelfTest extends GridCommonA
private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
/** Cache name. */
- private static final String CACHE = "cache";
+ protected static final String CACHE = "cache";
/** Empty cache name. */
private static final String EMPTY_CACHE = "emptyCache";
http://git-wip-us.apache.org/repos/asf/ignite/blob/ac8602db/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalFieldsQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalFieldsQuerySelfTest.java
index be1f196..462118f 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalFieldsQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalFieldsQuerySelfTest.java
@@ -18,6 +18,8 @@
package org.apache.ignite.internal.processors.cache.local;
import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.internal.IgniteEx;
import org.apache.ignite.internal.processors.cache.IgniteCacheAbstractFieldsQuerySelfTest;
import static org.apache.ignite.cache.CacheMode.LOCAL;
@@ -26,6 +28,10 @@ import static org.apache.ignite.cache.CacheMode.LOCAL;
* Tests for fields queries.
*/
public class IgniteCacheLocalFieldsQuerySelfTest extends IgniteCacheAbstractFieldsQuerySelfTest {
+// static {
+// System.setProperty(IgniteSystemProperties.IGNITE_H2_DEBUG_CONSOLE, "1");
+// }
+
/** {@inheritDoc} */
@Override protected CacheMode cacheMode() {
return LOCAL;
@@ -35,4 +41,14 @@ public class IgniteCacheLocalFieldsQuerySelfTest extends IgniteCacheAbstractFiel
@Override protected int gridCount() {
return 1;
}
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testInformationSchema() throws Exception {
+ IgniteEx ignite = grid(0);
+
+ ignite.cache(CACHE).query(
+ new SqlFieldsQuery("SELECT VALUE FROM INFORMATION_SCHEMA.SETTINGS").setLocal(true)).getAll();
+ }
}
\ No newline at end of file
[11/25] ignite git commit: IGNITE-4336 Manual rebalance can't be
requested twice
Posted by sb...@apache.org.
IGNITE-4336 Manual rebalance can't be requested twice
Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/c14561c3
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/c14561c3
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/c14561c3
Branch: refs/heads/ignite-4371
Commit: c14561c337cf145aced381de1f1ca9676733b862
Parents: 59e6fec
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Dec 8 17:58:54 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Dec 8 17:58:54 2016 +0300
----------------------------------------------------------------------
.../processors/cache/GridCacheAdapter.java | 6 +-
.../GridCachePartitionExchangeManager.java | 11 ++-
.../processors/cache/GridCachePreloader.java | 8 ++-
.../cache/GridCachePreloaderAdapter.java | 12 ++--
.../processors/cache/IgniteCacheProxy.java | 4 +-
.../dht/preloader/GridDhtPartitionDemander.java | 46 ++++++++++--
.../GridDhtPartitionsExchangeFuture.java | 16 ++++-
.../dht/preloader/GridDhtPreloader.java | 11 +--
.../cache/CacheRebalancingSelfTest.java | 75 ++++++++++++++++++++
.../testsuites/IgniteCacheTestSuite5.java | 9 ++-
10 files changed, 166 insertions(+), 32 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/c14561c3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 2e24e67..0f39c69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -84,8 +84,8 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
import org.apache.ignite.internal.processors.cache.affinity.GridCacheAffinityImpl;
import org.apache.ignite.internal.processors.cache.distributed.IgniteExternalizableExpiryPolicy;
import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo;
import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter;
@@ -4651,9 +4651,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
/** {@inheritDoc} */
@Override public IgniteInternalFuture<?> rebalance() {
- ctx.preloader().forcePreload();
-
- return ctx.preloader().syncFuture();
+ return ctx.preloader().forceRebalance();
}
/** {@inheritDoc} */
http://git-wip-us.apache.org/repos/asf/ignite/blob/c14561c3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index f04a6ce..8ea2169 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -711,9 +711,13 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
*
* @param exchFut Exchange future.
*/
- public void forcePreloadExchange(GridDhtPartitionsExchangeFuture exchFut) {
+ public IgniteInternalFuture<Boolean> forceRebalance(GridDhtPartitionsExchangeFuture exchFut) {
+ GridFutureAdapter<Boolean> fut = new GridFutureAdapter<>();
+
exchWorker.addFuture(
- new GridDhtPartitionsExchangeFuture(cctx, exchFut.discoveryEvent(), exchFut.exchangeId()));
+ new GridDhtPartitionsExchangeFuture(cctx, exchFut.discoveryEvent(), exchFut.exchangeId(), fut));
+
+ return fut;
}
/**
@@ -1771,7 +1775,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
Runnable cur = cacheCtx.preloader().addAssignments(assigns,
forcePreload,
cnt,
- r);
+ r,
+ exchFut.forcedRebalanceFuture());
if (cur != null) {
rebList.add(U.maskName(cacheCtx.name()));
http://git-wip-us.apache.org/repos/asf/ignite/blob/c14561c3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
index 3c4456d..0c28691 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessageV2;
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
import org.apache.ignite.lang.IgnitePredicate;
import org.jetbrains.annotations.Nullable;
@@ -90,7 +91,8 @@ public interface GridCachePreloader {
public Runnable addAssignments(GridDhtPreloaderAssignments assignments,
boolean forcePreload,
int cnt,
- Runnable next);
+ Runnable next,
+ @Nullable GridFutureAdapter<Boolean> forcedRebFut);
/**
* @param p Preload predicate.
@@ -150,9 +152,9 @@ public interface GridCachePreloader {
AffinityTopologyVersion topVer);
/**
- * Force preload process.
+ * Force Rebalance process.
*/
- public void forcePreload();
+ public IgniteInternalFuture<Boolean> forceRebalance();
/**
* Unwinds undeploys.
http://git-wip-us.apache.org/repos/asf/ignite/blob/c14561c3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
index 656a960..d7ec288 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments;
import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
import org.apache.ignite.lang.IgnitePredicate;
import org.jetbrains.annotations.Nullable;
@@ -88,8 +89,8 @@ public class GridCachePreloaderAdapter implements GridCachePreloader {
}
/** {@inheritDoc} */
- @Override public void forcePreload() {
- // No-op.
+ @Override public IgniteInternalFuture<Boolean> forceRebalance() {
+ return new GridFinishedFuture<>(true);
}
/** {@inheritDoc} */
@@ -165,8 +166,11 @@ public class GridCachePreloaderAdapter implements GridCachePreloader {
}
/** {@inheritDoc} */
- @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload,
- int cnt, Runnable next) {
+ @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments,
+ boolean forcePreload,
+ int cnt,
+ Runnable next,
+ @Nullable GridFutureAdapter<Boolean> forcedRebFut) {
return null;
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/c14561c3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index f87fa1d..f03a3b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -2326,9 +2326,7 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
/** {@inheritDoc} */
@Override public IgniteFuture<?> rebalance() {
- ctx.preloader().forcePreload();
-
- return new IgniteFutureImpl<>(ctx.preloader().syncFuture());
+ return new IgniteFutureImpl<>(ctx.preloader().forceRebalance());
}
/**
http://git-wip-us.apache.org/repos/asf/ignite/blob/c14561c3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index a6808c7..02c31da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -55,6 +55,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartit
import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter;
import org.apache.ignite.internal.util.GridLeanSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
import org.apache.ignite.internal.util.future.GridFutureAdapter;
import org.apache.ignite.internal.util.tostring.GridToStringExclude;
import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -64,6 +65,7 @@ import org.apache.ignite.internal.util.typedef.T2;
import org.apache.ignite.internal.util.typedef.internal.LT;
import org.apache.ignite.internal.util.typedef.internal.S;
import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
import org.apache.ignite.lang.IgnitePredicate;
import org.apache.ignite.lang.IgniteUuid;
import org.apache.ignite.spi.IgniteSpiException;
@@ -216,9 +218,9 @@ public class GridDhtPartitionDemander {
}
/**
- * Force preload.
+ * Force Rebalance.
*/
- void forcePreload() {
+ IgniteInternalFuture<Boolean> forceRebalance() {
GridTimeoutObject obj = lastTimeoutObj.getAndSet(null);
if (obj != null)
@@ -230,14 +232,31 @@ public class GridDhtPartitionDemander {
if (log.isDebugEnabled())
log.debug("Forcing rebalance event for future: " + exchFut);
+ final GridFutureAdapter<Boolean> fut = new GridFutureAdapter<>();
+
exchFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
@Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
- cctx.shared().exchange().forcePreloadExchange(exchFut);
+ IgniteInternalFuture<Boolean> fut0 = cctx.shared().exchange().forceRebalance(exchFut);
+
+ fut0.listen(new IgniteInClosure<IgniteInternalFuture<Boolean>>() {
+ @Override public void apply(IgniteInternalFuture<Boolean> future) {
+ try {
+ fut.onDone(future.get());
+ }
+ catch (Exception e) {
+ fut.onDone(e);
+ }
+ }
+ });
}
});
+
+ return fut;
}
else if (log.isDebugEnabled())
log.debug("Ignoring force rebalance request (no topology event happened yet).");
+
+ return new GridFinishedFuture<>(true);
}
/**
@@ -275,15 +294,19 @@ public class GridDhtPartitionDemander {
* @param force {@code True} if dummy reassign.
* @param cnt Counter.
* @param next Runnable responsible for cache rebalancing start.
+ * @param forcedRebFut External future for forced rebalance.
* @return Rebalancing runnable.
*/
Runnable addAssignments(final GridDhtPreloaderAssignments assigns,
boolean force,
int cnt,
- final Runnable next) {
+ final Runnable next,
+ @Nullable final GridFutureAdapter<Boolean> forcedRebFut) {
if (log.isDebugEnabled())
log.debug("Adding partition assignments: " + assigns);
+ assert force == (forcedRebFut != null);
+
long delay = cctx.config().getRebalanceDelay();
if (delay == 0 || force) {
@@ -301,6 +324,19 @@ public class GridDhtPartitionDemander {
});
}
+ if (forcedRebFut != null) {
+ fut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
+ @Override public void apply(IgniteInternalFuture<Boolean> future) {
+ try {
+ forcedRebFut.onDone(future.get());
+ }
+ catch (Exception e) {
+ forcedRebFut.onDone(e);
+ }
+ }
+ });
+ }
+
rebalanceFut = fut;
fut.sendRebalanceStartedEvent();
@@ -383,7 +419,7 @@ public class GridDhtPartitionDemander {
@Override public void onTimeout() {
exchFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
@Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> f) {
- cctx.shared().exchange().forcePreloadExchange(exchFut);
+ cctx.shared().exchange().forceRebalance(exchFut);
}
});
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/c14561c3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index e945de9..4f34401 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -54,7 +54,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext;
import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
@@ -65,7 +64,6 @@ import org.apache.ignite.internal.util.tostring.GridToStringExclude;
import org.apache.ignite.internal.util.tostring.GridToStringInclude;
import org.apache.ignite.internal.util.typedef.CI1;
import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.T2;
import org.apache.ignite.internal.util.typedef.internal.CU;
import org.apache.ignite.internal.util.typedef.internal.LT;
import org.apache.ignite.internal.util.typedef.internal.S;
@@ -194,6 +192,9 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
/** */
private boolean centralizedAff;
+ /** Forced Rebalance future. */
+ private GridFutureAdapter<Boolean> forcedRebFut;
+
/**
* Dummy future created to trigger reassignments if partition
* topology changed while preloading.
@@ -227,15 +228,17 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
* @param cctx Cache context.
* @param discoEvt Discovery event.
* @param exchId Exchange id.
+ * @param forcedRebFut Forced Rebalance future.
*/
public GridDhtPartitionsExchangeFuture(GridCacheSharedContext cctx, DiscoveryEvent discoEvt,
- GridDhtPartitionExchangeId exchId) {
+ GridDhtPartitionExchangeId exchId, GridFutureAdapter<Boolean> forcedRebFut) {
dummy = false;
forcePreload = true;
this.exchId = exchId;
this.discoEvt = discoEvt;
this.cctx = cctx;
+ this.forcedRebFut = forcedRebFut;
reassign = true;
@@ -398,6 +401,13 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
}
/**
+ * @return Forced Rebalance future.
+ */
+ @Nullable public GridFutureAdapter<Boolean> forcedRebalanceFuture() {
+ return forcedRebFut;
+ }
+
+ /**
* @return {@code true} if entered to busy state.
*/
private boolean enterBusy() {
http://git-wip-us.apache.org/repos/asf/ignite/blob/c14561c3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
index 692e7c0..41bc2fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
@@ -413,8 +413,11 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
/** {@inheritDoc} */
@Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments,
- boolean forcePreload, int cnt, Runnable next) {
- return demander.addAssignments(assignments, forcePreload, cnt, next);
+ boolean forceRebalance,
+ int cnt,
+ Runnable next,
+ @Nullable GridFutureAdapter<Boolean> forcedRebFut) {
+ return demander.addAssignments(assignments, forceRebalance, cnt, next, forcedRebFut);
}
/**
@@ -728,8 +731,8 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
}
/** {@inheritDoc} */
- @Override public void forcePreload() {
- demander.forcePreload();
+ @Override public IgniteInternalFuture<Boolean> forceRebalance() {
+ return demander.forceRebalance();
}
/** {@inheritDoc} */
http://git-wip-us.apache.org/repos/asf/ignite/blob/c14561c3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java
new file mode 100644
index 0000000..8d1f67a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Test for rebalancing.
+ */
+public class CacheRebalancingSelfTest extends GridCommonAbstractTest {
+ /** {@inheritDoc} */
+ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+ IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+ cfg.setCacheConfiguration(new CacheConfiguration());
+
+ return cfg;
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testRebalanceFuture() throws Exception {
+ IgniteEx ignite0 = startGrid(0);
+ startGrid(1);
+
+ IgniteCache<Object, Object> cache = ignite0.cache(null);
+
+ IgniteFuture fut1 = cache.rebalance();
+
+ fut1.get();
+
+ startGrid(2);
+
+ IgniteFuture fut2 = cache.rebalance();
+
+ assert internalFuture(fut2) != internalFuture(fut1);
+
+ fut2.get();
+ }
+
+ /**
+ * @param future Future.
+ * @return Internal future.
+ */
+ private static IgniteInternalFuture internalFuture(IgniteFuture future) {
+ assert future instanceof IgniteFutureImpl;
+
+ return ((IgniteFutureImpl)future).internalFuture();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/c14561c3/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
----------------------------------------------------------------------
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 7f0e23c..144aac6 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
@@ -19,18 +19,19 @@ package org.apache.ignite.testsuites;
import junit.framework.TestSuite;
import org.apache.ignite.internal.processors.cache.CacheNearReaderUpdateTest;
+import org.apache.ignite.internal.processors.cache.CacheRebalancingSelfTest;
import org.apache.ignite.internal.processors.cache.CacheSerializableTransactionsTest;
-import org.apache.ignite.internal.processors.cache.GridCacheSwapSpaceSpiConsistencySelfTest;
import org.apache.ignite.internal.processors.cache.EntryVersionConsistencyReadThroughTest;
+import org.apache.ignite.internal.processors.cache.GridCacheOffHeapCleanupTest;
+import org.apache.ignite.internal.processors.cache.GridCacheSwapSpaceSpiConsistencySelfTest;
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.GridCacheOffHeapCleanupTest;
import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentFairAffinityTest;
import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentNodeJoinValidationTest;
import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentTest;
-import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheSyncRebalanceModeSelfTest;
import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheTxIteratorSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheSyncRebalanceModeSelfTest;
import org.apache.ignite.internal.processors.cache.store.IgniteCacheWriteBehindNoUpdateSelfTest;
/**
@@ -62,6 +63,8 @@ public class IgniteCacheTestSuite5 extends TestSuite {
suite.addTestSuite(GridCacheOffHeapCleanupTest.class);
+ suite.addTestSuite(CacheRebalancingSelfTest.class);
+
return suite;
}
}
[25/25] ignite git commit: Merge remote-tracking branch
'remotes/origin/master' into ignite-4371
Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/master' into ignite-4371
# Conflicts:
# modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
# modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
# modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
# modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
# modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
# modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
# modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
# modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/37a76190
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/37a76190
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/37a76190
Branch: refs/heads/ignite-4371
Commit: 37a76190d1447d7fbfe15659b8c52d983daa3775
Parents: 697e2fb 27a81c8
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 9 14:27:09 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 9 14:27:09 2016 +0300
----------------------------------------------------------------------
.../jmh/misc/JmhIncrementBenchmark.java | 95 +++++
.../rendezvous/RendezvousAffinityFunction.java | 80 ++--
.../communication/GridIoMessageFactory.java | 12 +-
.../communication/IgniteIoTestMessage.java | 2 +-
.../discovery/GridDiscoveryManager.java | 118 +-----
.../processors/cache/GridCacheAdapter.java | 13 +-
.../processors/cache/GridCacheEntryEx.java | 4 +-
.../processors/cache/GridCacheMapEntry.java | 17 +-
.../GridCachePartitionExchangeManager.java | 17 +-
.../processors/cache/GridCachePreloader.java | 8 +-
.../cache/GridCachePreloaderAdapter.java | 12 +-
.../processors/cache/GridCacheUtils.java | 17 -
.../processors/cache/IgniteCacheProxy.java | 4 +-
.../binary/CacheObjectBinaryProcessorImpl.java | 3 +-
.../GridDistributedTxFinishResponse.java | 6 +
.../dht/GridClientPartitionTopology.java | 120 +++---
.../distributed/dht/GridDhtLockFuture.java | 23 +-
.../dht/GridDhtPartitionTopology.java | 28 +-
.../dht/GridDhtPartitionTopologyImpl.java | 284 ++++++++------
.../dht/atomic/GridDhtAtomicCache.java | 2 +-
.../dht/preloader/GridDhtPartitionDemander.java | 46 ++-
.../dht/preloader/GridDhtPartitionFullMap.java | 18 +-
.../GridDhtPartitionsExchangeFuture.java | 72 +++-
.../dht/preloader/GridDhtPreloader.java | 11 +-
.../cache/transactions/IgniteTxHandler.java | 2 +-
.../transactions/IgniteTxLocalAdapter.java | 2 +-
.../processors/hadoop/HadoopJobProperty.java | 7 +
.../hadoop/HadoopMapperAwareTaskOutput.java | 32 ++
.../processors/hadoop/HadoopTaskInfo.java | 43 ++
.../shuffle/HadoopDirectShuffleMessage.java | 243 ++++++++++++
.../platform/PlatformAbstractTarget.java | 268 ++-----------
.../platform/PlatformAsyncTarget.java | 44 +++
.../platform/PlatformNoopProcessor.java | 41 +-
.../processors/platform/PlatformProcessor.java | 42 +-
.../platform/PlatformProcessorImpl.java | 87 +++--
.../processors/platform/PlatformTarget.java | 103 +++--
.../platform/PlatformTargetProxy.java | 126 ++++++
.../platform/PlatformTargetProxyImpl.java | 222 +++++++++++
.../binary/PlatformBinaryProcessor.java | 6 +-
.../platform/cache/PlatformCache.java | 15 +-
.../platform/cache/PlatformCacheIterator.java | 2 +-
.../cache/affinity/PlatformAffinity.java | 4 +-
.../affinity/PlatformAffinityFunction.java | 7 +-
.../PlatformAffinityFunctionTarget.java | 4 +-
.../query/PlatformAbstractQueryCursor.java | 4 +-
.../query/PlatformContinuousQueryProxy.java | 3 +-
.../callback/PlatformCallbackGateway.java | 6 +-
.../callback/PlatformCallbackUtils.java | 6 +-
.../platform/cluster/PlatformClusterGroup.java | 18 +-
.../platform/compute/PlatformCompute.java | 15 +-
.../datastreamer/PlatformDataStreamer.java | 4 +-
.../PlatformStreamReceiverImpl.java | 8 +-
.../datastructures/PlatformAtomicLong.java | 4 +-
.../datastructures/PlatformAtomicReference.java | 8 +-
.../datastructures/PlatformAtomicSequence.java | 2 +-
.../platform/events/PlatformEvents.java | 15 +-
.../platform/messaging/PlatformMessaging.java | 9 +-
.../platform/services/PlatformServices.java | 27 +-
.../transactions/PlatformTransactions.java | 8 +-
.../platform/utils/PlatformFutureUtils.java | 14 +-
.../utils/PlatformListenableTarget.java | 62 +++
.../service/GridServiceProcessor.java | 4 +-
.../internal/visor/query/VisorQueryJob.java | 2 +-
.../communication/tcp/TcpCommunicationSpi.java | 4 +-
.../ignite/spi/discovery/tcp/ServerImpl.java | 41 +-
.../tcp/internal/TcpDiscoveryStatistics.java | 4 +
.../AbstractAffinityFunctionSelfTest.java | 2 +-
.../GridDiscoveryManagerAliveCacheSelfTest.java | 2 +-
.../cache/CacheRebalancingSelfTest.java | 75 ++++
.../GridCachePartitionedAffinitySpreadTest.java | 7 +-
.../processors/cache/GridCacheTestEntryEx.java | 3 +-
.../distributed/dht/GridCacheDhtTestUtils.java | 232 -----------
...eCacheExpiryPolicyWithStoreAbstractTest.java | 40 ++
.../testsuites/IgniteCacheTestSuite5.java | 9 +-
.../processors/hadoop/HadoopMapperUtils.java | 56 +++
.../hadoop/impl/v2/HadoopV2Context.java | 11 +
.../hadoop/impl/v2/HadoopV2MapTask.java | 10 +
.../hadoop/jobtracker/HadoopJobTracker.java | 4 +
.../hadoop/shuffle/HadoopShuffle.java | 23 +-
.../hadoop/shuffle/HadoopShuffleJob.java | 389 ++++++++++++++-----
.../shuffle/HadoopShuffleRemoteState.java | 5 +-
.../shuffle/direct/HadoopDirectDataInput.java | 166 ++++++++
.../shuffle/direct/HadoopDirectDataOutput.java | 221 +++++++++++
.../direct/HadoopDirectDataOutputContext.java | 100 +++++
.../direct/HadoopDirectDataOutputState.java | 54 +++
.../child/HadoopChildProcessRunner.java | 2 +-
.../impl/HadoopMapReduceEmbeddedSelfTest.java | 22 +-
.../h2/twostep/GridReduceQueryExecutor.java | 14 +-
.../cpp/jni/include/ignite/jni/exports.h | 3 -
.../platforms/cpp/jni/include/ignite/jni/java.h | 7 -
modules/platforms/cpp/jni/project/vs/module.def | 2 -
modules/platforms/cpp/jni/src/exports.cpp | 8 -
modules/platforms/cpp/jni/src/java.cpp | 76 ++--
.../Services/ServicesTest.cs | 9 +-
.../Apache.Ignite.Core.csproj | 1 +
.../Apache.Ignite.Core/Impl/Common/Future.cs | 13 +-
.../Impl/Common/Listenable.cs | 49 +++
.../Impl/Compute/ComputeImpl.cs | 4 +-
.../Apache.Ignite.Core/Impl/PlatformTarget.cs | 2 +-
.../Impl/Unmanaged/IgniteJniNativeMethods.cs | 8 -
.../Impl/Unmanaged/UnmanagedUtils.cs | 5 -
101 files changed, 2904 insertions(+), 1310 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/37a76190/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/37a76190/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/37a76190/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/37a76190/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
[02/25] ignite git commit: ignite-4296 Optimize
GridDhtPartitionsSingleMessage processing - optimized processing of
GridDhtPartitionsSingleMessage - minor optimizations for
RendezvousAffinityFunction - fixed heartbeats sending in tcp discovery
Posted by sb...@apache.org.
ignite-4296 Optimize GridDhtPartitionsSingleMessage processing
- optimized processing of GridDhtPartitionsSingleMessage
- minor optimizations for RendezvousAffinityFunction
- fixed heartbeats sending in tcp discovery
Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/acf20b32
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/acf20b32
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/acf20b32
Branch: refs/heads/ignite-4371
Commit: acf20b32d8fb68e42b904b091fb2b943f4558cef
Parents: b4aedfd
Author: sboikov <sb...@gridgain.com>
Authored: Mon Dec 5 14:01:28 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Dec 5 14:01:28 2016 +0300
----------------------------------------------------------------------
.../rendezvous/RendezvousAffinityFunction.java | 80 ++++--
.../discovery/GridDiscoveryManager.java | 118 +-------
.../GridCachePartitionExchangeManager.java | 6 +-
.../processors/cache/GridCacheUtils.java | 17 --
.../binary/CacheObjectBinaryProcessorImpl.java | 3 +-
.../dht/GridClientPartitionTopology.java | 120 ++++----
.../distributed/dht/GridDhtLocalPartition.java | 1 -
.../dht/GridDhtPartitionTopology.java | 28 +-
.../dht/GridDhtPartitionTopologyImpl.java | 284 +++++++++++--------
.../dht/preloader/GridDhtPartitionFullMap.java | 18 +-
.../GridDhtPartitionsExchangeFuture.java | 56 +++-
.../cache/transactions/IgniteTxHandler.java | 4 +-
.../service/GridServiceProcessor.java | 4 +-
.../ignite/spi/discovery/tcp/ServerImpl.java | 40 ++-
.../tcp/internal/TcpDiscoveryStatistics.java | 4 +
.../AbstractAffinityFunctionSelfTest.java | 2 +-
.../GridDiscoveryManagerAliveCacheSelfTest.java | 2 +-
.../GridCachePartitionedAffinitySpreadTest.java | 7 +-
.../distributed/dht/GridCacheDhtTestUtils.java | 232 ---------------
.../h2/twostep/GridReduceQueryExecutor.java | 14 +-
20 files changed, 437 insertions(+), 603 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
index ec12973..75e7c92 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
@@ -17,7 +17,6 @@
package org.apache.ignite.cache.affinity.rendezvous;
-import java.io.ByteArrayOutputStream;
import java.io.Externalizable;
import java.io.IOException;
import java.io.ObjectInput;
@@ -354,46 +353,69 @@ public class RendezvousAffinityFunction implements AffinityFunction, Externaliza
/**
* Returns collection of nodes (primary first) for specified partition.
+ *
+ * @param d Message digest.
+ * @param part Partition.
+ * @param nodes Nodes.
+ * @param nodesHash Serialized nodes hashes.
+ * @param backups Number of backups.
+ * @param neighborhoodCache Neighborhood.
+ * @return Assignment.
*/
- public List<ClusterNode> assignPartition(int part, List<ClusterNode> nodes, int backups,
+ public List<ClusterNode> assignPartition(MessageDigest d,
+ int part,
+ List<ClusterNode> nodes,
+ Map<ClusterNode, byte[]> nodesHash,
+ int backups,
@Nullable Map<UUID, Collection<ClusterNode>> neighborhoodCache) {
if (nodes.size() <= 1)
return nodes;
- List<IgniteBiTuple<Long, ClusterNode>> lst = new ArrayList<>();
+ if (d == null)
+ d = digest.get();
- MessageDigest d = digest.get();
+ List<IgniteBiTuple<Long, ClusterNode>> lst = new ArrayList<>(nodes.size());
- for (ClusterNode node : nodes) {
- Object nodeHash = resolveNodeHash(node);
+ try {
+ for (int i = 0; i < nodes.size(); i++) {
+ ClusterNode node = nodes.get(i);
- try {
- ByteArrayOutputStream out = new ByteArrayOutputStream();
+ byte[] nodeHashBytes = nodesHash.get(node);
- byte[] nodeHashBytes = U.marshal(ignite.configuration().getMarshaller(), nodeHash);
+ if (nodeHashBytes == null) {
+ Object nodeHash = resolveNodeHash(node);
- out.write(U.intToBytes(part), 0, 4); // Avoid IOException.
- out.write(nodeHashBytes, 0, nodeHashBytes.length); // Avoid IOException.
+ byte[] nodeHashBytes0 = U.marshal(ignite.configuration().getMarshaller(), nodeHash);
+
+ // Add 4 bytes for partition bytes.
+ nodeHashBytes = new byte[nodeHashBytes0.length + 4];
+
+ System.arraycopy(nodeHashBytes0, 0, nodeHashBytes, 4, nodeHashBytes0.length);
+
+ nodesHash.put(node, nodeHashBytes);
+ }
+
+ U.intToBytes(part, nodeHashBytes, 0);
d.reset();
- byte[] bytes = d.digest(out.toByteArray());
+ byte[] bytes = d.digest(nodeHashBytes);
long hash =
- (bytes[0] & 0xFFL)
- | ((bytes[1] & 0xFFL) << 8)
- | ((bytes[2] & 0xFFL) << 16)
- | ((bytes[3] & 0xFFL) << 24)
- | ((bytes[4] & 0xFFL) << 32)
- | ((bytes[5] & 0xFFL) << 40)
- | ((bytes[6] & 0xFFL) << 48)
- | ((bytes[7] & 0xFFL) << 56);
+ (bytes[0] & 0xFFL)
+ | ((bytes[1] & 0xFFL) << 8)
+ | ((bytes[2] & 0xFFL) << 16)
+ | ((bytes[3] & 0xFFL) << 24)
+ | ((bytes[4] & 0xFFL) << 32)
+ | ((bytes[5] & 0xFFL) << 40)
+ | ((bytes[6] & 0xFFL) << 48)
+ | ((bytes[7] & 0xFFL) << 56);
lst.add(F.t(hash, node));
}
- catch (IgniteCheckedException e) {
- throw new IgniteException(e);
- }
+ }
+ catch (IgniteCheckedException e) {
+ throw new IgniteException(e);
}
Collections.sort(lst, COMPARATOR);
@@ -474,8 +496,18 @@ public class RendezvousAffinityFunction implements AffinityFunction, Externaliza
Map<UUID, Collection<ClusterNode>> neighborhoodCache = exclNeighbors ?
GridCacheUtils.neighbors(affCtx.currentTopologySnapshot()) : null;
+ MessageDigest d = digest.get();
+
+ List<ClusterNode> nodes = affCtx.currentTopologySnapshot();
+
+ Map<ClusterNode, byte[]> nodesHash = U.newHashMap(nodes.size());
+
for (int i = 0; i < parts; i++) {
- List<ClusterNode> partAssignment = assignPartition(i, affCtx.currentTopologySnapshot(), affCtx.backups(),
+ List<ClusterNode> partAssignment = assignPartition(d,
+ i,
+ nodes,
+ nodesHash,
+ affCtx.backups(),
neighborhoodCache);
assignments.add(partAssignment);
http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index d24f900..ddd4ee3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -42,7 +42,7 @@ import java.util.UUID;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.LinkedBlockingQueue;
@@ -1623,17 +1623,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
/**
* Gets cache remote nodes for cache with given name.
*
- * @param cacheName Cache name.
- * @param topVer Topology version.
- * @return Collection of cache nodes.
- */
- public Collection<ClusterNode> remoteCacheNodes(@Nullable String cacheName, AffinityTopologyVersion topVer) {
- return resolveDiscoCache(cacheName, topVer).remoteCacheNodes(cacheName, topVer.topologyVersion());
- }
-
- /**
- * Gets cache remote nodes for cache with given name.
- *
* @param topVer Topology version.
* @return Collection of cache nodes.
*/
@@ -1648,7 +1637,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
* @param topVer Topology version.
* @return Collection of cache nodes.
*/
- public Collection<ClusterNode> aliveCacheNodes(@Nullable String cacheName, AffinityTopologyVersion topVer) {
+ Collection<ClusterNode> aliveCacheNodes(@Nullable String cacheName, AffinityTopologyVersion topVer) {
return resolveDiscoCache(cacheName, topVer).aliveCacheNodes(cacheName, topVer.topologyVersion());
}
@@ -1659,38 +1648,20 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
* @param topVer Topology version.
* @return Collection of cache nodes.
*/
- public Collection<ClusterNode> aliveRemoteCacheNodes(@Nullable String cacheName, AffinityTopologyVersion topVer) {
+ Collection<ClusterNode> aliveRemoteCacheNodes(@Nullable String cacheName, AffinityTopologyVersion topVer) {
return resolveDiscoCache(cacheName, topVer).aliveRemoteCacheNodes(cacheName, topVer.topologyVersion());
}
/**
- * Gets alive remote server nodes with at least one cache configured.
- *
* @param topVer Topology version (maximum allowed node order).
- * @return Collection of alive cache nodes.
+ * @return Oldest alive server nodes with at least one cache configured.
*/
- public Collection<ClusterNode> aliveRemoteServerNodesWithCaches(AffinityTopologyVersion topVer) {
- return resolveDiscoCache(null, topVer).aliveRemoteServerNodesWithCaches(topVer.topologyVersion());
- }
+ @Nullable public ClusterNode oldestAliveCacheServerNode(AffinityTopologyVersion topVer) {
+ DiscoCache cache = resolveDiscoCache(null, topVer);
- /**
- * Gets alive server nodes with at least one cache configured.
- *
- * @param topVer Topology version (maximum allowed node order).
- * @return Collection of alive cache nodes.
- */
- public Collection<ClusterNode> aliveServerNodesWithCaches(AffinityTopologyVersion topVer) {
- return resolveDiscoCache(null, topVer).aliveServerNodesWithCaches(topVer.topologyVersion());
- }
+ Map.Entry<ClusterNode, Boolean> e = cache.aliveSrvNodesWithCaches.firstEntry();
- /**
- * Gets alive nodes with at least one cache configured.
- *
- * @param topVer Topology version (maximum allowed node order).
- * @return Collection of alive cache nodes.
- */
- public Collection<ClusterNode> aliveNodesWithCaches(AffinityTopologyVersion topVer) {
- return resolveDiscoCache(null, topVer).aliveNodesWithCaches(topVer.topologyVersion());
+ return e != null ? e.getKey() : null;
}
/**
@@ -2580,19 +2551,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
private final ConcurrentMap<String, Collection<ClusterNode>> aliveRmtCacheNodes;
/**
- * Cached alive remote nodes with caches.
- */
- private final Collection<ClusterNode> aliveNodesWithCaches;
-
- /**
* Cached alive server remote nodes with caches.
*/
- private final Collection<ClusterNode> aliveSrvNodesWithCaches;
-
- /**
- * Cached alive remote server nodes with caches.
- */
- private final Collection<ClusterNode> aliveRmtSrvNodesWithCaches;
+ private final ConcurrentSkipListMap<ClusterNode, Boolean> aliveSrvNodesWithCaches;
/**
* @param loc Local node.
@@ -2625,9 +2586,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
aliveCacheNodes = new ConcurrentHashMap8<>(allNodes.size(), 1.0f);
aliveRmtCacheNodes = new ConcurrentHashMap8<>(allNodes.size(), 1.0f);
- aliveNodesWithCaches = new ConcurrentSkipListSet<>();
- aliveSrvNodesWithCaches = new ConcurrentSkipListSet<>();
- aliveRmtSrvNodesWithCaches = new ConcurrentSkipListSet<>();
+ aliveSrvNodesWithCaches = new ConcurrentSkipListMap<>(GridNodeOrderComparator.INSTANCE);
nodesByVer = new TreeMap<>();
long maxOrder0 = 0;
@@ -2681,18 +2640,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
}
}
- if (hasCaches) {
- if (alive(node.id())) {
- aliveNodesWithCaches.add(node);
-
- if (!CU.clientNode(node)) {
- aliveSrvNodesWithCaches.add(node);
-
- if (!loc.id().equals(node.id()))
- aliveRmtSrvNodesWithCaches.add(node);
- }
- }
- }
+ if (hasCaches && alive(node.id()) && !CU.clientNode(node))
+ aliveSrvNodesWithCaches.put(node, Boolean.TRUE);
IgniteProductVersion nodeVer = U.productVersion(node);
@@ -2821,17 +2770,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
}
/**
- * Gets all remote nodes that have cache with given name.
- *
- * @param cacheName Cache name.
- * @param topVer Topology version.
- * @return Collection of nodes.
- */
- Collection<ClusterNode> remoteCacheNodes(@Nullable String cacheName, final long topVer) {
- return filter(topVer, rmtCacheNodes.get(cacheName));
- }
-
- /**
* Gets all remote nodes that have at least one cache configured.
*
* @param topVer Topology version.
@@ -2876,36 +2814,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
}
/**
- * Gets all alive remote server nodes with at least one cache configured.
- *
- * @param topVer Topology version.
- * @return Collection of nodes.
- */
- Collection<ClusterNode> aliveRemoteServerNodesWithCaches(final long topVer) {
- return filter(topVer, aliveRmtSrvNodesWithCaches);
- }
-
- /**
- * Gets all alive server nodes with at least one cache configured.
- *
- * @param topVer Topology version.
- * @return Collection of nodes.
- */
- Collection<ClusterNode> aliveServerNodesWithCaches(final long topVer) {
- return filter(topVer, aliveSrvNodesWithCaches);
- }
-
- /**
- * Gets all alive remote nodes with at least one cache configured.
- *
- * @param topVer Topology version.
- * @return Collection of nodes.
- */
- Collection<ClusterNode> aliveNodesWithCaches(final long topVer) {
- return filter(topVer, aliveNodesWithCaches);
- }
-
- /**
* Checks if cache with given name has at least one node with near cache enabled.
*
* @param cacheName Cache name.
@@ -2928,9 +2836,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
filterNodeMap(aliveRmtCacheNodes, leftNode);
- aliveNodesWithCaches.remove(leftNode);
aliveSrvNodesWithCaches.remove(leftNode);
- aliveRmtSrvNodesWithCaches.remove(leftNode);
}
/**
http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 503b334..7a24aa1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -740,7 +740,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
* Partition refresh callback.
*/
private void refreshPartitions() {
- ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, AffinityTopologyVersion.NONE);
+ ClusterNode oldest = cctx.discovery().oldestAliveCacheServerNode(AffinityTopologyVersion.NONE);
if (oldest == null) {
if (log.isDebugEnabled())
@@ -1224,7 +1224,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
top = cacheCtx.topology();
if (top != null)
- updated |= top.update(null, entry.getValue(), null) != null;
+ updated |= top.update(null, entry.getValue(), null);
}
if (!cctx.kernalContext().clientNode() && updated)
@@ -1273,7 +1273,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
top = cacheCtx.topology();
if (top != null) {
- updated |= top.update(null, entry.getValue(), null) != null;
+ updated |= top.update(null, entry.getValue(), null, true);
cctx.affinity().checkRebalanceState(top, cacheId);
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 90e428c..d32f4c1 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
@@ -490,23 +490,6 @@ public class GridCacheUtils {
}
/**
- * Gets oldest alive server node with at least one cache configured for specified topology version.
- *
- * @param ctx Context.
- * @param topVer Maximum allowed topology version.
- * @return Oldest alive cache server node.
- */
- @Nullable public static ClusterNode oldestAliveCacheServerNode(GridCacheSharedContext ctx,
- AffinityTopologyVersion topVer) {
- Collection<ClusterNode> nodes = ctx.discovery().aliveServerNodesWithCaches(topVer);
-
- if (nodes.isEmpty())
- return null;
-
- return oldest(nodes);
- }
-
- /**
* @param nodes Nodes.
* @return Oldest node for the given topology version.
*/
http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
index 568a4da..1d60c42 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
@@ -304,8 +304,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
assert !metaDataCache.context().affinityNode();
while (true) {
- ClusterNode oldestSrvNode =
- CU.oldestAliveCacheServerNode(ctx.cache().context(), AffinityTopologyVersion.NONE);
+ ClusterNode oldestSrvNode = ctx.discovery().oldestAliveCacheServerNode(AffinityTopologyVersion.NONE);
if (oldestSrvNode == null)
break;
http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
index 5efb317..816132d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
@@ -271,7 +271,7 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
removeNode(exchId.nodeId());
// In case if node joins, get topology at the time of joining node.
- ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, topVer);
+ ClusterNode oldest = cctx.discovery().oldestAliveCacheServerNode(topVer);
assert oldest != null;
@@ -536,7 +536,8 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
try {
assert node2part != null && node2part.valid() : "Invalid node2part [node2part: " + node2part +
- ", locNodeId=" + cctx.localNodeId() + ", gridName=" + cctx.gridName() + ']';
+ ", locNodeId=" + cctx.localNodeId() +
+ ", gridName=" + cctx.gridName() + ']';
GridDhtPartitionFullMap m = node2part;
@@ -549,7 +550,7 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
/** {@inheritDoc} */
@SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"})
- @Nullable @Override public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId,
+ @Nullable @Override public boolean update(@Nullable GridDhtPartitionExchangeId exchId,
GridDhtPartitionFullMap partMap,
Map<Integer, Long> cntrMap) {
if (log.isDebugEnabled())
@@ -563,7 +564,7 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
log.debug("Stale exchange id for full partition map update (will ignore) [lastExchId=" +
lastExchangeId + ", exchId=" + exchId + ']');
- return null;
+ return false;
}
if (node2part != null && node2part.compareTo(partMap) >= 0) {
@@ -571,7 +572,7 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
log.debug("Stale partition map for full partition map update (will ignore) [lastExchId=" +
lastExchangeId + ", exchId=" + exchId + ", curMap=" + node2part + ", newMap=" + partMap + ']');
- return null;
+ return false;
}
updateSeq.incrementAndGet();
@@ -634,7 +635,7 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
if (log.isDebugEnabled())
log.debug("Partition map after full update: " + fullMapString());
- return null;
+ return false;
}
finally {
lock.writeLock().unlock();
@@ -642,10 +643,10 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
}
/** {@inheritDoc} */
- @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"})
- @Nullable @Override public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId,
+ @Nullable @Override public boolean update(@Nullable GridDhtPartitionExchangeId exchId,
GridDhtPartitionMap2 parts,
- Map<Integer, Long> cntrMap) {
+ Map<Integer, Long> cntrMap,
+ boolean checkEvictions) {
if (log.isDebugEnabled())
log.debug("Updating single partition map [exchId=" + exchId + ", parts=" + mapString(parts) + ']');
@@ -654,29 +655,27 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
log.debug("Received partition update for non-existing node (will ignore) [exchId=" + exchId +
", parts=" + parts + ']');
- return null;
+ return false;
}
lock.writeLock().lock();
try {
if (stopping)
- return null;
+ return false;
if (lastExchangeId != null && exchId != null && lastExchangeId.compareTo(exchId) > 0) {
if (log.isDebugEnabled())
log.debug("Stale exchange id for single partition map update (will ignore) [lastExchId=" +
lastExchangeId + ", exchId=" + exchId + ']');
- return null;
+ return false;
}
if (exchId != null)
lastExchangeId = exchId;
if (node2part == null) {
- U.dumpStack(log, "Created invalid: " + node2part);
-
// Create invalid partition map.
node2part = new GridDhtPartitionFullMap();
}
@@ -688,43 +687,45 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
log.debug("Stale update sequence for single partition map update (will ignore) [exchId=" + exchId +
", curSeq=" + cur.updateSequence() + ", newSeq=" + parts.updateSequence() + ']');
- return null;
+ return false;
}
long updateSeq = this.updateSeq.incrementAndGet();
- node2part = new GridDhtPartitionFullMap(node2part, updateSeq);
-
- boolean changed = false;
+ node2part.updateSequence(updateSeq);
- if (cur == null || !cur.equals(parts))
- changed = true;
+ boolean changed = cur == null || !cur.equals(parts);
- node2part.put(parts.nodeId(), parts);
+ if (changed) {
+ node2part.put(parts.nodeId(), parts);
- part2node = new HashMap<>(part2node);
+ // Add new mappings.
+ for (Integer p : parts.keySet()) {
+ Set<UUID> ids = part2node.get(p);
- // Add new mappings.
- for (Integer p : parts.keySet()) {
- Set<UUID> ids = part2node.get(p);
+ if (ids == null)
+ // Initialize HashSet to size 3 in anticipation that there won't be
+ // more than 3 nodes per partition.
+ part2node.put(p, ids = U.newHashSet(3));
- if (ids == null)
- // Initialize HashSet to size 3 in anticipation that there won't be
- // more than 3 nodes per partition.
- part2node.put(p, ids = U.newHashSet(3));
+ ids.add(parts.nodeId());
+ }
- changed |= ids.add(parts.nodeId());
- }
+ // Remove obsolete mappings.
+ if (cur != null) {
+ for (Integer p : cur.keySet()) {
+ if (parts.containsKey(p))
+ continue;
- // Remove obsolete mappings.
- if (cur != null) {
- for (Integer p : F.view(cur.keySet(), F0.notIn(parts.keySet()))) {
- Set<UUID> ids = part2node.get(p);
+ Set<UUID> ids = part2node.get(p);
- if (ids != null)
- changed |= ids.remove(parts.nodeId());
+ if (ids != null)
+ ids.remove(parts.nodeId());
+ }
}
}
+ else
+ cur.updateSequence(parts.updateSequence(), parts.topologyVersion());
if (cntrMap != null) {
for (Map.Entry<Integer, Long> e : cntrMap.entrySet()) {
@@ -740,13 +741,18 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
if (log.isDebugEnabled())
log.debug("Partition map after single update: " + fullMapString());
- return changed ? localPartitionMap() : null;
+ return changed;
}
finally {
lock.writeLock().unlock();
}
}
+ /** {@inheritDoc} */
+ @Override public void checkEvictions() {
+ // No-op.
+ }
+
/**
* Updates value for single partition.
*
@@ -755,13 +761,12 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
* @param state State.
* @param updateSeq Update sequence.
*/
- @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"})
private void updateLocal(int p, UUID nodeId, GridDhtPartitionState state, long updateSeq) {
assert lock.isWriteLockedByCurrentThread();
assert nodeId.equals(cctx.localNodeId());
// In case if node joins, get topology at the time of joining node.
- ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, topVer);
+ ClusterNode oldest = cctx.discovery().oldestAliveCacheServerNode(topVer);
// If this node became the oldest node.
if (oldest.id().equals(cctx.localNodeId())) {
@@ -811,7 +816,7 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
assert nodeId != null;
assert lock.writeLock().isHeldByCurrentThread();
- ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, topVer);
+ ClusterNode oldest = cctx.discovery().oldestAliveCacheServerNode(topVer);
ClusterNode loc = cctx.localNode();
@@ -877,18 +882,6 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
}
/** {@inheritDoc} */
- @Nullable @Override public GridDhtPartitionMap2 partitions(UUID nodeId) {
- lock.readLock().lock();
-
- try {
- return node2part.get(nodeId);
- }
- finally {
- lock.readLock().unlock();
- }
- }
-
- /** {@inheritDoc} */
@Override public Map<Integer, Long> updateCounters(boolean skipZeros) {
lock.readLock().lock();
@@ -919,6 +912,27 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
}
/** {@inheritDoc} */
+ @Override public boolean hasMovingPartitions() {
+ lock.readLock().lock();
+
+ try {
+ assert node2part != null && node2part.valid() : "Invalid node2part [node2part: " + node2part +
+ ", locNodeId=" + cctx.localNodeId() +
+ ", gridName=" + cctx.gridName() + ']';
+
+ for (GridDhtPartitionMap2 map : node2part.values()) {
+ if (map.hasMovingPartitions())
+ return true;
+ }
+
+ return false;
+ }
+ finally {
+ lock.readLock().unlock();
+ }
+ }
+
+ /** {@inheritDoc} */
@Override public void printMemoryStats(int threshold) {
X.println(">>> Cache partition topology stats [grid=" + cctx.gridName() + ", cacheId=" + cacheId + ']');
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index 39a3e08..668a1cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -479,7 +479,6 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
if ((reservations & 0xFFFF) == 0 && casState(reservations, RENTING)) {
shouldBeRenting = false;
-
if (log.isDebugEnabled())
log.debug("Moved partition to RENTING state: " + this);
http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
index 0f75a5d..14ce1f9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
@@ -195,6 +195,11 @@ public interface GridDhtPartitionTopology {
public GridDhtPartitionFullMap partitionMap(boolean onlyActive);
/**
+ * @return {@code True} If one of cache nodes has partitions in {@link GridDhtPartitionState#MOVING} state.
+ */
+ public boolean hasMovingPartitions();
+
+ /**
* @param e Entry removed from cache.
*/
public void onRemoved(GridDhtCacheEntry e);
@@ -203,9 +208,9 @@ public interface GridDhtPartitionTopology {
* @param exchId Exchange ID.
* @param partMap Update partition map.
* @param cntrMap Partition update counters.
- * @return Local partition map if there were evictions or {@code null} otherwise.
+ * @return {@code True} if topology state changed.
*/
- public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId,
+ public boolean update(@Nullable GridDhtPartitionExchangeId exchId,
GridDhtPartitionFullMap partMap,
@Nullable Map<Integer, Long> cntrMap);
@@ -213,11 +218,18 @@ public interface GridDhtPartitionTopology {
* @param exchId Exchange ID.
* @param parts Partitions.
* @param cntrMap Partition update counters.
- * @return Local partition map if there were evictions or {@code null} otherwise.
+ * @param checkEvictions Check evictions flag.
+ * @return {@code True} if topology state changed.
*/
- @Nullable public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId,
+ @Nullable public boolean update(@Nullable GridDhtPartitionExchangeId exchId,
GridDhtPartitionMap2 parts,
- @Nullable Map<Integer, Long> cntrMap);
+ @Nullable Map<Integer, Long> cntrMap,
+ boolean checkEvictions);
+
+ /**
+ *
+ */
+ public void checkEvictions();
/**
* @param skipZeros If {@code true} then filters out zero counters.
@@ -238,12 +250,6 @@ public interface GridDhtPartitionTopology {
public void onEvicted(GridDhtLocalPartition part, boolean updateSeq);
/**
- * @param nodeId Node to get partitions for.
- * @return Partitions for node.
- */
- @Nullable public GridDhtPartitionMap2 partitions(UUID nodeId);
-
- /**
* Prints memory stats.
*
* @param threshold Threshold for number of entries.
http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index ab573bd..1b4dcc9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -44,7 +44,6 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap;
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap2;
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
-import org.apache.ignite.internal.util.F0;
import org.apache.ignite.internal.util.GridAtomicLong;
import org.apache.ignite.internal.util.StripedCompositeReadWriteLock;
import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -330,8 +329,9 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
}
/** {@inheritDoc} */
- @Override public void initPartitions(
- GridDhtPartitionsExchangeFuture exchFut) throws IgniteInterruptedCheckedException {
+ @Override public void initPartitions(GridDhtPartitionsExchangeFuture exchFut)
+ throws IgniteInterruptedCheckedException
+ {
U.writeLock(lock);
try {
@@ -356,9 +356,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
private void initPartitions0(GridDhtPartitionsExchangeFuture exchFut, long updateSeq) {
ClusterNode loc = cctx.localNode();
- ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx.shared(), topVer);
-
- assert oldest != null || cctx.kernalContext().clientNode();
+ ClusterNode oldest = currentCoordinator();
GridDhtPartitionExchangeId exchId = exchFut.exchangeId();
@@ -397,7 +395,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
if (log.isDebugEnabled())
log.debug("Owned partition for oldest node: " + locPart);
- updateLocal(p, loc.id(), locPart.state(), updateSeq);
+ updateSeq = updateLocal(p, locPart.state(), updateSeq);
}
}
}
@@ -419,7 +417,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
if (state.active()) {
locPart.rent(false);
- updateLocal(p, loc.id(), locPart.state(), updateSeq);
+ updateSeq = updateLocal(p, locPart.state(), updateSeq);
if (log.isDebugEnabled())
log.debug("Evicting partition with rebalancing disabled " +
@@ -443,8 +441,6 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
* @param updateSeq Update sequence.
*/
private void createPartitions(List<List<ClusterNode>> aff, long updateSeq) {
- ClusterNode loc = cctx.localNode();
-
int num = cctx.affinity().partitions();
for (int p = 0; p < num; p++) {
@@ -454,7 +450,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
// will be created in MOVING state.
GridDhtLocalPartition locPart = createPartition(p);
- updateLocal(p, loc.id(), locPart.state(), updateSeq);
+ updateSeq = updateLocal(p, locPart.state(), updateSeq);
}
}
// If this node's map is empty, we pre-create local partitions,
@@ -485,10 +481,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
if (exchId.isLeft())
removeNode(exchId.nodeId());
- // In case if node joins, get topology at the time of joining node.
- ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx.shared(), topVer);
-
- assert oldest != null || cctx.kernalContext().clientNode();
+ ClusterNode oldest = currentCoordinator();
if (log.isDebugEnabled())
log.debug("Partition map beforeExchange [exchId=" + exchId + ", fullMap=" + fullMapString() + ']');
@@ -548,8 +541,6 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
@Override public boolean afterExchange(GridDhtPartitionsExchangeFuture exchFut) throws IgniteCheckedException {
boolean changed = waitForRent();
- ClusterNode loc = cctx.localNode();
-
int num = cctx.affinity().partitions();
AffinityTopologyVersion topVer = exchFut.topologyVersion();
@@ -600,7 +591,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
assert owned : "Failed to own partition [cacheName" + cctx.name() + ", locPart=" +
locPart + ']';
- updateLocal(p, loc.id(), locPart.state(), updateSeq);
+ updateSeq = updateLocal(p, locPart.state(), updateSeq);
changed = true;
@@ -620,7 +611,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
locPart + ", owners = " + owners + ']');
}
else
- updateLocal(p, loc.id(), locPart.state(), updateSeq);
+ updateSeq = updateLocal(p, locPart.state(), updateSeq);
}
}
else {
@@ -630,7 +621,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
if (state == MOVING) {
locPart.rent(false);
- updateLocal(p, loc.id(), locPart.state(), updateSeq);
+ updateSeq = updateLocal(p, locPart.state(), updateSeq);
changed = true;
@@ -803,8 +794,11 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
map.put(i, part.state());
}
- return new GridDhtPartitionMap2(cctx.nodeId(), updateSeq.get(), topVer,
- Collections.unmodifiableMap(map), true);
+ return new GridDhtPartitionMap2(cctx.nodeId(),
+ updateSeq.get(),
+ topVer,
+ Collections.unmodifiableMap(map),
+ true);
}
finally {
lock.readLock().unlock();
@@ -985,7 +979,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
/** {@inheritDoc} */
@SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"})
- @Nullable @Override public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId,
+ @Nullable @Override public boolean update(@Nullable GridDhtPartitionExchangeId exchId,
GridDhtPartitionFullMap partMap,
@Nullable Map<Integer, Long> cntrMap) {
if (log.isDebugEnabled())
@@ -997,7 +991,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
try {
if (stopping)
- return null;
+ return false;
if (cntrMap != null) {
for (Map.Entry<Integer, Long> e : cntrMap.entrySet()) {
@@ -1025,7 +1019,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
log.debug("Stale exchange id for full partition map update (will ignore) [lastExchId=" +
lastExchangeId + ", exchId=" + exchId + ']');
- return null;
+ return false;
}
if (node2part != null && node2part.compareTo(partMap) >= 0) {
@@ -1033,7 +1027,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
log.debug("Stale partition map for full partition map update (will ignore) [lastExchId=" +
lastExchangeId + ", exchId=" + exchId + ", curMap=" + node2part + ", newMap=" + partMap + ']');
- return null;
+ return false;
}
long updateSeq = this.updateSeq.incrementAndGet();
@@ -1076,7 +1070,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
node2part = partMap;
- Map<Integer, Set<UUID>> p2n = new HashMap<>(cctx.affinity().partitions(), 1.0f);
+ Map<Integer, Set<UUID>> p2n = U.newHashMap(cctx.affinity().partitions());
for (Map.Entry<UUID, GridDhtPartitionMap2> e : partMap.entrySet()) {
for (Integer p : e.getValue().keySet()) {
@@ -1110,7 +1104,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
if (log.isDebugEnabled())
log.debug("Partition map after full update: " + fullMapString());
- return changed ? localPartitionMap() : null;
+ return changed;
}
finally {
lock.writeLock().unlock();
@@ -1118,10 +1112,10 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
}
/** {@inheritDoc} */
- @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"})
- @Nullable @Override public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId,
+ @Nullable @Override public boolean update(@Nullable GridDhtPartitionExchangeId exchId,
GridDhtPartitionMap2 parts,
- @Nullable Map<Integer, Long> cntrMap) {
+ @Nullable Map<Integer, Long> cntrMap,
+ boolean checkEvictions) {
if (log.isDebugEnabled())
log.debug("Updating single partition map [exchId=" + exchId + ", parts=" + mapString(parts) + ']');
@@ -1130,33 +1124,28 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
log.debug("Received partition update for non-existing node (will ignore) [exchId=" + exchId +
", parts=" + parts + ']');
- return null;
+ return false;
}
lock.writeLock().lock();
try {
if (stopping)
- return null;
+ return false;
if (cntrMap != null) {
for (Map.Entry<Integer, Long> e : cntrMap.entrySet()) {
- Long cntr = this.cntrMap.get(e.getKey());
+ Integer p = e.getKey();
- if (cntr == null || cntr < e.getValue())
- this.cntrMap.put(e.getKey(), e.getValue());
- }
-
- for (int i = 0; i < locParts.length(); i++) {
- GridDhtLocalPartition part = locParts.get(i);
+ Long cntr = this.cntrMap.get(p);
- if (part == null)
- continue;
+ if (cntr == null || cntr < e.getValue())
+ this.cntrMap.put(p, e.getValue());
- Long cntr = cntrMap.get(part.id());
+ GridDhtLocalPartition part = locParts.get(p);
- if (cntr != null)
- part.updateCounter(cntr);
+ if (part != null)
+ part.updateCounter(e.getValue());
}
}
@@ -1165,7 +1154,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
log.debug("Stale exchange id for single partition map update (will ignore) [lastExchId=" +
lastExchangeId + ", exchId=" + exchId + ']');
- return null;
+ return false;
}
if (exchId != null)
@@ -1182,60 +1171,91 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
log.debug("Stale update sequence for single partition map update (will ignore) [exchId=" + exchId +
", curSeq=" + cur.updateSequence() + ", newSeq=" + parts.updateSequence() + ']');
- return null;
+ return false;
}
long updateSeq = this.updateSeq.incrementAndGet();
- node2part = new GridDhtPartitionFullMap(node2part, updateSeq);
-
- boolean changed = false;
+ node2part.newUpdateSequence(updateSeq);
- if (cur == null || !cur.equals(parts))
- changed = true;
+ boolean changed = cur == null || !cur.equals(parts);
- node2part.put(parts.nodeId(), parts);
+ if (changed) {
+ node2part.put(parts.nodeId(), parts);
- part2node = new HashMap<>(part2node);
+ // Add new mappings.
+ for (Integer p : parts.keySet()) {
+ Set<UUID> ids = part2node.get(p);
- // Add new mappings.
- for (Integer p : parts.keySet()) {
- Set<UUID> ids = part2node.get(p);
+ if (ids == null)
+ // Initialize HashSet to size 3 in anticipation that there won't be
+ // more than 3 nodes per partition.
+ part2node.put(p, ids = U.newHashSet(3));
- if (ids == null)
- // Initialize HashSet to size 3 in anticipation that there won't be
- // more than 3 nodes per partition.
- part2node.put(p, ids = U.newHashSet(3));
+ ids.add(parts.nodeId());
+ }
- changed |= ids.add(parts.nodeId());
- }
+ // Remove obsolete mappings.
+ if (cur != null) {
+ for (Integer p : cur.keySet()) {
+ if (parts.containsKey(p))
+ continue;
- // Remove obsolete mappings.
- if (cur != null) {
- for (Integer p : F.view(cur.keySet(), F0.notIn(parts.keySet()))) {
- Set<UUID> ids = part2node.get(p);
+ Set<UUID> ids = part2node.get(p);
- if (ids != null)
- changed |= ids.remove(parts.nodeId());
+ if (ids != null)
+ ids.remove(parts.nodeId());
+ }
}
}
+ else
+ cur.updateSequence(parts.updateSequence(), parts.topologyVersion());
- AffinityTopologyVersion affVer = cctx.affinity().affinityTopologyVersion();
-
- if (!affVer.equals(AffinityTopologyVersion.NONE) && affVer.compareTo(topVer) >= 0) {
- List<List<ClusterNode>> aff = cctx.affinity().assignments(topVer);
-
- changed |= checkEvictions(updateSeq, aff);
-
- updateRebalanceVersion(aff);
- }
+ if (checkEvictions)
+ changed |= checkEvictions(updateSeq);
consistencyCheck();
if (log.isDebugEnabled())
log.debug("Partition map after single update: " + fullMapString());
- return changed ? localPartitionMap() : null;
+ return changed;
+ }
+ finally {
+ lock.writeLock().unlock();
+ }
+ }
+
+ /**
+ * @param updateSeq Update sequence.
+ * @return {@code True} if state changed.
+ */
+ private boolean checkEvictions(long updateSeq) {
+ AffinityTopologyVersion affVer = cctx.affinity().affinityTopologyVersion();
+
+ boolean changed = false;
+
+ if (!affVer.equals(AffinityTopologyVersion.NONE) && affVer.compareTo(topVer) >= 0) {
+ List<List<ClusterNode>> aff = cctx.affinity().assignments(topVer);
+
+ changed = checkEvictions(updateSeq, aff);
+
+ updateRebalanceVersion(aff);
+ }
+
+ return changed;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void checkEvictions() {
+ lock.writeLock().lock();
+
+ try {
+ long updateSeq = this.updateSeq.incrementAndGet();
+
+ node2part.newUpdateSequence(updateSeq);
+
+ checkEvictions(updateSeq);
}
finally {
lock.writeLock().unlock();
@@ -1270,7 +1290,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
if (nodeIds.containsAll(F.nodeIds(affNodes))) {
part.rent(false);
- updateLocal(part.id(), locId, part.state(), updateSeq);
+ updateSeq = updateLocal(part.id(), part.state(), updateSeq);
changed = true;
@@ -1295,7 +1315,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
if (locId.equals(n.id())) {
part.rent(false);
- updateLocal(part.id(), locId, part.state(), updateSeq);
+ updateSeq = updateLocal(part.id(), part.state(), updateSeq);
changed = true;
@@ -1316,19 +1336,27 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
}
/**
+ * @return Current coordinator node.
+ */
+ @Nullable private ClusterNode currentCoordinator() {
+ ClusterNode oldest = cctx.discovery().oldestAliveCacheServerNode(topVer);
+
+ assert oldest != null || cctx.kernalContext().clientNode();
+
+ return oldest;
+ }
+
+ /**
* Updates value for single partition.
*
* @param p Partition.
- * @param nodeId Node ID.
* @param state State.
* @param updateSeq Update sequence.
+ * @return Update sequence.
*/
@SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"})
- private void updateLocal(int p, UUID nodeId, GridDhtPartitionState state, long updateSeq) {
- assert nodeId.equals(cctx.nodeId());
-
- // In case if node joins, get topology at the time of joining node.
- ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx.shared(), topVer);
+ private long updateLocal(int p, GridDhtPartitionState state, long updateSeq) {
+ ClusterNode oldest = currentCoordinator();
assert oldest != null || cctx.kernalContext().clientNode();
@@ -1338,12 +1366,16 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
if (seq != updateSeq) {
if (seq > updateSeq) {
- if (this.updateSeq.get() < seq) {
+ long seq0 = this.updateSeq.get();
+
+ if (seq0 < seq) {
// Update global counter if necessary.
- boolean b = this.updateSeq.compareAndSet(this.updateSeq.get(), seq + 1);
+ boolean b = this.updateSeq.compareAndSet(seq0, seq + 1);
- assert b : "Invalid update sequence [updateSeq=" + updateSeq + ", seq=" + seq +
- ", curUpdateSeq=" + this.updateSeq.get() + ", node2part=" + node2part.toFullString() + ']';
+ assert b : "Invalid update sequence [updateSeq=" + updateSeq +
+ ", seq=" + seq +
+ ", curUpdateSeq=" + this.updateSeq.get() +
+ ", node2part=" + node2part.toFullString() + ']';
updateSeq = seq + 1;
}
@@ -1355,11 +1387,19 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
}
}
- GridDhtPartitionMap2 map = node2part.get(nodeId);
+ UUID locNodeId = cctx.localNodeId();
+
+ GridDhtPartitionMap2 map = node2part.get(locNodeId);
- if (map == null)
- node2part.put(nodeId, map = new GridDhtPartitionMap2(nodeId, updateSeq, topVer,
- Collections.<Integer, GridDhtPartitionState>emptyMap(), false));
+ if (map == null) {
+ map = new GridDhtPartitionMap2(locNodeId,
+ updateSeq,
+ topVer,
+ Collections.<Integer, GridDhtPartitionState>emptyMap(),
+ false);
+
+ node2part.put(locNodeId, map);
+ }
map.updateSequence(updateSeq, topVer);
@@ -1370,7 +1410,9 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
if (ids == null)
part2node.put(p, ids = U.newHashSet(3));
- ids.add(nodeId);
+ ids.add(locNodeId);
+
+ return updateSeq;
}
/**
@@ -1395,8 +1437,6 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
else
node2part = new GridDhtPartitionFullMap(node2part, node2part.updateSequence());
- part2node = new HashMap<>(part2node);
-
GridDhtPartitionMap2 parts = node2part.remove(nodeId);
if (parts != null) {
@@ -1418,13 +1458,11 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
/** {@inheritDoc} */
@Override public boolean own(GridDhtLocalPartition part) {
- ClusterNode loc = cctx.localNode();
-
lock.writeLock().lock();
try {
if (part.own()) {
- updateLocal(part.id(), loc.id(), part.state(), updateSeq.incrementAndGet());
+ updateLocal(part.id(), part.state(), updateSeq.incrementAndGet());
consistencyCheck();
@@ -1452,7 +1490,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
long seq = updateSeq ? this.updateSeq.incrementAndGet() : this.updateSeq.get();
- updateLocal(part.id(), cctx.localNodeId(), part.state(), seq);
+ updateLocal(part.id(), part.state(), seq);
consistencyCheck();
}
@@ -1462,18 +1500,6 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
}
/** {@inheritDoc} */
- @Nullable @Override public GridDhtPartitionMap2 partitions(UUID nodeId) {
- lock.readLock().lock();
-
- try {
- return node2part.get(nodeId);
- }
- finally {
- lock.readLock().unlock();
- }
- }
-
- /** {@inheritDoc} */
@Override public Map<Integer, Long> updateCounters(boolean skipZeros) {
lock.readLock().lock();
@@ -1526,6 +1552,30 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
}
/** {@inheritDoc} */
+ @Override public boolean hasMovingPartitions() {
+ lock.readLock().lock();
+
+ try {
+ assert node2part != null && node2part.valid() : "Invalid node2part [node2part: " + node2part +
+ ", cache=" + cctx.name() +
+ ", started=" + cctx.started() +
+ ", stopping=" + stopping +
+ ", locNodeId=" + cctx.localNode().id() +
+ ", locName=" + cctx.gridName() + ']';
+
+ for (GridDhtPartitionMap2 map : node2part.values()) {
+ if (map.hasMovingPartitions())
+ return true;
+ }
+
+ return false;
+ }
+ finally {
+ lock.readLock().unlock();
+ }
+ }
+
+ /** {@inheritDoc} */
@Override public void printMemoryStats(int threshold) {
X.println(">>> Cache partition topology stats [grid=" + cctx.gridName() + ", cache=" + cctx.name() + ']');
@@ -1607,10 +1657,12 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
if (state == match)
return true;
- if (matches != null && matches.length > 0)
- for (GridDhtPartitionState s : matches)
+ if (matches != null && matches.length > 0) {
+ for (GridDhtPartitionState s : matches) {
if (state == s)
return true;
+ }
+ }
}
return false;
http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java
index 8f5ad17..e8860f2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java
@@ -103,10 +103,13 @@ public class GridDhtPartitionFullMap extends HashMap<UUID, GridDhtPartitionMap2>
for (Map.Entry<UUID, GridDhtPartitionMap2> e : m.entrySet()) {
GridDhtPartitionMap2 part = e.getValue();
- if (onlyActive)
- put(e.getKey(), new GridDhtPartitionMap2(part.nodeId(), part.updateSequence(), part.topologyVersion(), part.map(), true));
- else
- put(e.getKey(), part);
+ GridDhtPartitionMap2 cpy = new GridDhtPartitionMap2(part.nodeId(),
+ part.updateSequence(),
+ part.topologyVersion(),
+ part.map(),
+ onlyActive);
+
+ put(e.getKey(), cpy);
}
}
@@ -177,6 +180,13 @@ public class GridDhtPartitionFullMap extends HashMap<UUID, GridDhtPartitionMap2>
/**
* @param updateSeq New update sequence value.
+ */
+ public void newUpdateSequence(long updateSeq) {
+ this.updateSeq = updateSeq;
+ }
+
+ /**
+ * @param updateSeq New update sequence value.
* @return Old update sequence value.
*/
public long updateSequence(long updateSeq) {
http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index f391265..e945de9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -112,6 +112,10 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
/** */
@GridToStringExclude
+ private int pendingSingleUpdates;
+
+ /** */
+ @GridToStringExclude
private List<ClusterNode> srvNodes;
/** */
@@ -1162,13 +1166,16 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
*/
private void processMessage(ClusterNode node, GridDhtPartitionsSingleMessage msg) {
boolean allReceived = false;
+ boolean updateSingleMap = false;
synchronized (mux) {
assert crd != null;
if (crd.isLocal()) {
if (remaining.remove(node.id())) {
- updatePartitionSingleMap(msg);
+ updateSingleMap = true;
+
+ pendingSingleUpdates++;
allReceived = remaining.isEmpty();
}
@@ -1177,8 +1184,42 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
singleMsgs.put(node, msg);
}
- if (allReceived)
+ if (updateSingleMap) {
+ try {
+ updatePartitionSingleMap(msg);
+ }
+ finally {
+ synchronized (mux) {
+ assert pendingSingleUpdates > 0;
+
+ pendingSingleUpdates--;
+
+ if (pendingSingleUpdates == 0)
+ mux.notifyAll();
+ }
+ }
+ }
+
+ if (allReceived) {
+ awaitSingleMapUpdates();
+
onAllReceived(false);
+ }
+ }
+
+ /**
+ *
+ */
+ private void awaitSingleMapUpdates() {
+ synchronized (mux) {
+ try {
+ while (pendingSingleUpdates > 0)
+ U.wait(mux);
+ }
+ catch (IgniteInterruptedCheckedException e) {
+ U.warn(log, "Failed to wait for partition map updates, thread was interrupted: " + e);
+ }
+ }
}
/**
@@ -1218,6 +1259,11 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
}
}
+ for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
+ if (!cacheCtx.isLocal())
+ cacheCtx.topology().checkEvictions();
+ }
+
updateLastVersion(cctx.versions().last());
cctx.versions().onExchange(lastVer.get().order());
@@ -1374,7 +1420,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
if (cacheCtx != null)
cacheCtx.topology().update(exchId, entry.getValue(), cntrMap);
else {
- ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, AffinityTopologyVersion.NONE);
+ ClusterNode oldest = cctx.discovery().oldestAliveCacheServerNode(AffinityTopologyVersion.NONE);
if (oldest != null && oldest.isLocal())
cctx.exchange().clientTopology(cacheId, this).update(exchId, entry.getValue(), cntrMap);
@@ -1395,7 +1441,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
GridDhtPartitionTopology top = cacheCtx != null ? cacheCtx.topology() :
cctx.exchange().clientTopology(cacheId, this);
- top.update(exchId, entry.getValue(), msg.partitionUpdateCounters(cacheId));
+ top.update(exchId, entry.getValue(), msg.partitionUpdateCounters(cacheId), false);
}
}
@@ -1557,6 +1603,8 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
if (crd0.isLocal()) {
if (allReceived) {
+ awaitSingleMapUpdates();
+
onAllReceived(true);
return;
http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index fbd8ce5..cf69264 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -679,14 +679,14 @@ public class IgniteTxHandler {
* @param req Request.
* @return Future.
*/
- @Nullable public IgniteInternalFuture<IgniteInternalTx> processNearTxFinishRequest(UUID nodeId,
+ @Nullable private IgniteInternalFuture<IgniteInternalTx> processNearTxFinishRequest(UUID nodeId,
GridNearTxFinishRequest req) {
if (txFinishMsgLog.isDebugEnabled())
txFinishMsgLog.debug("Received near finish request [txId=" + req.version() + ", node=" + nodeId + ']');
IgniteInternalFuture<IgniteInternalTx> fut = finish(nodeId, null, req);
- assert req.txState() != null || fut.error() != null ||
+ assert req.txState() != null || (fut != null && fut.error() != null) ||
(ctx.tm().tx(req.version()) == null && ctx.tm().nearTx(req.version()) == null);
return fut;
http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index 6c26363..b9b92b8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -1265,7 +1265,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
try {
if (!cache.context().affinityNode()) {
ClusterNode oldestSrvNode =
- CU.oldestAliveCacheServerNode(cache.context().shared(), AffinityTopologyVersion.NONE);
+ ctx.discovery().oldestAliveCacheServerNode(AffinityTopologyVersion.NONE);
if (oldestSrvNode == null)
return new GridEmptyIterator<>();
@@ -1589,7 +1589,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
depExe.submit(new BusyRunnable() {
@Override public void run0() {
- ClusterNode oldest = CU.oldestAliveCacheServerNode(cache.context().shared(), topVer);
+ ClusterNode oldest = ctx.discovery().oldestAliveCacheServerNode(topVer);
if (oldest != null && oldest.isLocal()) {
final Collection<GridServiceDeployment> retries = new ConcurrentLinkedQueue<>();
http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 8814745..a660ec8 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -129,6 +129,7 @@ import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingRequest;
import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingResponse;
import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryRedirectToClient;
import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusCheckMessage;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
import org.jetbrains.annotations.Nullable;
import org.jsr166.ConcurrentHashMap8;
@@ -174,8 +175,7 @@ class ServerImpl extends TcpDiscoveryImpl {
IgniteProductVersion.fromString("1.5.0");
/** */
- private final ThreadPoolExecutor utilityPool = new ThreadPoolExecutor(0, 1, 2000, TimeUnit.MILLISECONDS,
- new LinkedBlockingQueue<Runnable>());
+ private IgniteThreadPoolExecutor utilityPool;
/** Nodes ring. */
@GridToStringExclude
@@ -297,6 +297,13 @@ class ServerImpl extends TcpDiscoveryImpl {
spiState = DISCONNECTED;
}
+ utilityPool = new IgniteThreadPoolExecutor("disco-pool",
+ spi.ignite().name(),
+ 0,
+ 1,
+ 2000,
+ new LinkedBlockingQueue<Runnable>());
+
if (debugMode) {
if (!log.isInfoEnabled())
throw new IgniteSpiException("Info log level should be enabled for TCP discovery to work " +
@@ -2403,9 +2410,12 @@ class ServerImpl extends TcpDiscoveryImpl {
/** Connection check threshold. */
private long connCheckThreshold;
+ /** */
+ private long lastRingMsgTime;
+
/**
*/
- protected RingMessageWorker() {
+ RingMessageWorker() {
super("tcp-disco-msg-worker", 10);
initConnectionCheckFrequency();
@@ -2500,6 +2510,8 @@ class ServerImpl extends TcpDiscoveryImpl {
* @param msg Message to process.
*/
@Override protected void processMessage(TcpDiscoveryAbstractMessage msg) {
+ sendHeartbeatMessage();
+
DebugLogger log = messageLogger(msg);
if (log.isDebugEnabled())
@@ -2508,6 +2520,11 @@ class ServerImpl extends TcpDiscoveryImpl {
if (debugMode)
debugLog(msg, "Processing message [cls=" + msg.getClass().getSimpleName() + ", id=" + msg.id() + ']');
+ boolean ensured = spi.ensured(msg);
+
+ if (!locNode.id().equals(msg.senderNodeId()) && ensured)
+ lastRingMsgTime = U.currentTimeMillis();
+
if (locNode.internalOrder() == 0) {
boolean proc = false;
@@ -2564,7 +2581,7 @@ class ServerImpl extends TcpDiscoveryImpl {
else
assert false : "Unknown message type: " + msg.getClass().getSimpleName();
- if (spi.ensured(msg) && redirectToClients(msg))
+ if (ensured && redirectToClients(msg))
msgHist.add(msg);
if (msg.senderNodeId() != null && !msg.senderNodeId().equals(getLocalNodeId())) {
@@ -5336,12 +5353,9 @@ class ServerImpl extends TcpDiscoveryImpl {
* Sends heartbeat message if needed.
*/
private void sendHeartbeatMessage() {
- if (!isLocalNodeCoordinator())
- return;
-
long elapsed = (lastTimeHbMsgSent + spi.hbFreq) - U.currentTimeMillis();
- if (elapsed > 0)
+ if (elapsed > 0 || !isLocalNodeCoordinator())
return;
TcpDiscoveryHeartbeatMessage msg = new TcpDiscoveryHeartbeatMessage(getConfiguredNodeId());
@@ -5361,7 +5375,9 @@ class ServerImpl extends TcpDiscoveryImpl {
if (lastTimeStatusMsgSent < locNode.lastUpdateTime())
lastTimeStatusMsgSent = locNode.lastUpdateTime();
- long elapsed = (lastTimeStatusMsgSent + hbCheckFreq) - U.currentTimeMillis();
+ long updateTime = Math.max(lastTimeStatusMsgSent, lastRingMsgTime);
+
+ long elapsed = (updateTime + hbCheckFreq) - U.currentTimeMillis();
if (elapsed > 0)
return;
@@ -6062,11 +6078,11 @@ class ServerImpl extends TcpDiscoveryImpl {
TcpDiscoverySpiState state = spiStateCopy();
- long socketTimeout = spi.failureDetectionTimeoutEnabled() ? spi.failureDetectionTimeout() :
+ long sockTimeout = spi.failureDetectionTimeoutEnabled() ? spi.failureDetectionTimeout() :
spi.getSocketTimeout();
if (state == CONNECTED) {
- spi.writeToSocket(msg, sock, RES_OK, socketTimeout);
+ spi.writeToSocket(msg, sock, RES_OK, sockTimeout);
if (log.isDebugEnabled())
log.debug("Responded to join request message [msg=" + msg + ", res=" + RES_OK + ']');
@@ -6103,7 +6119,7 @@ class ServerImpl extends TcpDiscoveryImpl {
// Local node is stopping. Remote node should try next one.
res = RES_CONTINUE_JOIN;
- spi.writeToSocket(msg, sock, res, socketTimeout);
+ spi.writeToSocket(msg, sock, res, sockTimeout);
if (log.isDebugEnabled())
log.debug("Responded to join request message [msg=" + msg + ", res=" + res + ']');
http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java
index 9e73632..c790644 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java
@@ -28,6 +28,8 @@ import org.apache.ignite.internal.util.typedef.internal.S;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.lang.IgniteUuid;
import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAbstractMessage;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCustomEventMessage;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddFinishedMessage;
import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddedMessage;
import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeFailedMessage;
import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeLeftMessage;
@@ -317,7 +319,9 @@ public class TcpDiscoveryStatistics {
assert time >= 0 : time;
if (crdSinceTs.get() > 0 &&
+ (msg instanceof TcpDiscoveryCustomEventMessage) ||
(msg instanceof TcpDiscoveryNodeAddedMessage) ||
+ (msg instanceof TcpDiscoveryNodeAddFinishedMessage) ||
(msg instanceof TcpDiscoveryNodeLeftMessage) ||
(msg instanceof TcpDiscoveryNodeFailedMessage)) {
ringMsgsSndTs.put(msg.id(), U.currentTimeMillis());
http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
index 878d7d1..43017db 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
@@ -104,6 +104,7 @@ public abstract class AbstractAffinityFunctionSelfTest extends GridCommonAbstrac
}
/**
+ * @param backups Number of backups.
* @throws Exception If failed.
*/
protected void checkNodeRemoved(int backups) throws Exception {
@@ -247,7 +248,6 @@ public abstract class AbstractAffinityFunctionSelfTest extends GridCommonAbstrac
}
}
-
/**
* @param assignment Assignment to verify.
*/
http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
index 390c83e..31b4bc7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
@@ -239,7 +239,7 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
GridCacheSharedContext<?, ?> ctx = k.context().cache().context();
ClusterNode oldest =
- GridCacheUtils.oldestAliveCacheServerNode(ctx, new AffinityTopologyVersion(currVer));
+ ctx.discovery().oldestAliveCacheServerNode(new AffinityTopologyVersion(currVer));
assertNotNull(oldest);
http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedAffinitySpreadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedAffinitySpreadTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedAffinitySpreadTest.java
index a59ca8b..2d46cf4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedAffinitySpreadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedAffinitySpreadTest.java
@@ -76,7 +76,12 @@ public class GridCachePartitionedAffinitySpreadTest extends GridCommonAbstractTe
Map<ClusterNode, Integer> parts = new HashMap<>(nodes.size());
for (int part = 0; part < aff.getPartitions(); part++) {
- Collection<ClusterNode> affNodes = aff.assignPartition(part, new ArrayList(nodes), 0, null);
+ Collection<ClusterNode> affNodes = aff.assignPartition(null,
+ part,
+ new ArrayList<>(nodes),
+ new HashMap<ClusterNode, byte[]>(),
+ 0,
+ null);
assertEquals(1, affNodes.size());
[22/25] ignite git commit: Added benchmark for atomic increments w/
and w/o thread-local batching.
Posted by sb...@apache.org.
Added benchmark for atomic increments w/ and w/o thread-local batching.
Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/b65802b9
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/b65802b9
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/b65802b9
Branch: refs/heads/ignite-4371
Commit: b65802b9d32d6c52bcf35b3a52a39b12ea34ec0a
Parents: 065ca4a
Author: devozerov <vo...@gridgain.com>
Authored: Fri Dec 9 13:10:22 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Dec 9 13:10:22 2016 +0300
----------------------------------------------------------------------
.../jmh/misc/JmhIncrementBenchmark.java | 95 ++++++++++++++++++++
1 file changed, 95 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/b65802b9/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/misc/JmhIncrementBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/misc/JmhIncrementBenchmark.java b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/misc/JmhIncrementBenchmark.java
new file mode 100644
index 0000000..629fdd3
--- /dev/null
+++ b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/misc/JmhIncrementBenchmark.java
@@ -0,0 +1,95 @@
+/*
+ * 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.benchmarks.jmh.misc;
+
+import org.apache.ignite.internal.benchmarks.jmh.JmhAbstractBenchmark;
+import org.apache.ignite.internal.benchmarks.jmh.runner.JmhIdeBenchmarkRunner;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.State;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Test various increment strategies.
+ */
+@State(Scope.Benchmark)
+public class JmhIncrementBenchmark extends JmhAbstractBenchmark {
+ /** Batch size. */
+ private static final int BATCH_SIZE = 1024;
+
+ /** Thread-local value. */
+ private static final ThreadLocal<Long> VAL = new ThreadLocal<Long>() {
+ @Override protected Long initialValue() {
+ return 0L;
+ }
+ };
+
+ /** Atomic variable. */
+ private final AtomicLong atomic = new AtomicLong();
+
+ /**
+ * Test standard increment.
+ */
+ @Benchmark
+ public long increment() {
+ return atomic.incrementAndGet();
+ }
+
+ /**
+ * Test increment with thread-local batching.
+ */
+ @Benchmark
+ public long threadLocalIncrement() {
+ long val = VAL.get();
+
+ if ((val & (BATCH_SIZE - 1)) == 0)
+ val = atomic.addAndGet(BATCH_SIZE);
+
+ VAL.set(++val);
+
+ return val;
+ }
+
+ /**
+ * Run benchmarks.
+ *
+ * @param args Arguments.
+ * @throws Exception If failed.
+ */
+ public static void main(String[] args) throws Exception {
+ run(8);
+ }
+
+ /**
+ * Run benchmark.
+ *
+ * @param threads Amount of threads.
+ * @throws Exception If failed.
+ */
+ private static void run(int threads) throws Exception {
+ JmhIdeBenchmarkRunner.create()
+ .forks(1)
+ .threads(threads)
+ .warmupIterations(30)
+ .measurementIterations(30)
+ .benchmarks(JmhIncrementBenchmark.class.getSimpleName())
+ .jvmArguments("-Xms4g", "-Xmx4g")
+ .run();
+ }
+}
[18/25] ignite git commit: IGNITE-3220 I/O bottleneck on
server/client cluster configuration Communications optimizations: -
possibility to open separate in/out connections - possibility to have
multiple connections between nodes - implemented NIO sessio
Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
index 63c9845..66f9176 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
@@ -20,9 +20,11 @@ package org.apache.ignite.internal.util.nio;
import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
import java.nio.channels.SelectionKey;
+import java.util.ArrayList;
import java.util.Collection;
+import java.util.List;
import java.util.concurrent.Semaphore;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.ignite.IgniteLogger;
import org.apache.ignite.internal.util.tostring.GridToStringExclude;
import org.apache.ignite.internal.util.typedef.internal.LT;
@@ -37,17 +39,14 @@ import org.jsr166.ConcurrentLinkedDeque8;
*/
class GridSelectorNioSessionImpl extends GridNioSessionImpl {
/** Pending write requests. */
- private final ConcurrentLinkedDeque8<GridNioFuture<?>> queue = new ConcurrentLinkedDeque8<>();
+ private final ConcurrentLinkedDeque8<SessionWriteRequest> queue = new ConcurrentLinkedDeque8<>();
/** Selection key associated with this session. */
@GridToStringExclude
private SelectionKey key;
- /** Worker index for server */
- private final int selectorIdx;
-
- /** Size counter. */
- private final AtomicInteger queueSize = new AtomicInteger();
+ /** Current worker thread. */
+ private volatile GridNioWorker worker;
/** Semaphore. */
@GridToStringExclude
@@ -59,17 +58,29 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
/** Read buffer. */
private ByteBuffer readBuf;
- /** Recovery data. */
- private GridNioRecoveryDescriptor recovery;
+ /** Incoming recovery data. */
+ private GridNioRecoveryDescriptor inRecovery;
+
+ /** Outgoing recovery data. */
+ private GridNioRecoveryDescriptor outRecovery;
/** Logger. */
private final IgniteLogger log;
+ /** */
+ private List<GridNioServer.SessionChangeRequest> pendingStateChanges;
+
+ /** */
+ final AtomicBoolean procWrite = new AtomicBoolean();
+
+ /** */
+ private Object sysMsg;
+
/**
* Creates session instance.
*
* @param log Logger.
- * @param selectorIdx Selector index for this session.
+ * @param worker NIO worker thread.
* @param filterChain Filter chain that will handle requests.
* @param locAddr Local address.
* @param rmtAddr Remote address.
@@ -80,7 +91,7 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
*/
GridSelectorNioSessionImpl(
IgniteLogger log,
- int selectorIdx,
+ GridNioWorker worker,
GridNioFilterChain filterChain,
InetSocketAddress locAddr,
InetSocketAddress rmtAddr,
@@ -91,7 +102,7 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
) {
super(filterChain, locAddr, rmtAddr, accepted);
- assert selectorIdx >= 0;
+ assert worker != null;
assert sndQueueLimit >= 0;
assert locAddr != null : "GridSelectorNioSessionImpl should have local socket address.";
@@ -101,7 +112,7 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
this.log = log;
- this.selectorIdx = selectorIdx;
+ this.worker = worker;
sem = sndQueueLimit > 0 ? new Semaphore(sndQueueLimit) : null;
@@ -119,12 +130,19 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
}
/**
+ * @return Worker.
+ */
+ GridNioWorker worker() {
+ return worker;
+ }
+
+ /**
* Sets selection key for this session.
*
* @param key Selection key.
*/
void key(SelectionKey key) {
- assert this.key == null;
+ assert key != null;
this.key = key;
}
@@ -151,10 +169,88 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
}
/**
- * @return Selector index.
+ * @param from Current session worker.
+ * @param fut Move future.
+ * @return {@code True} if session move was scheduled.
+ */
+ boolean offerMove(GridNioWorker from, GridNioServer.SessionChangeRequest fut) {
+ synchronized (this) {
+ if (log.isDebugEnabled())
+ log.debug("Offered move [ses=" + this + ", fut=" + fut + ']');
+
+ GridNioWorker worker0 = worker;
+
+ if (worker0 != from)
+ return false;
+
+ worker.offer(fut);
+ }
+
+ return true;
+ }
+
+ /**
+ * @param fut Future.
+ */
+ void offerStateChange(GridNioServer.SessionChangeRequest fut) {
+ synchronized (this) {
+ if (log.isDebugEnabled())
+ log.debug("Offered move [ses=" + this + ", fut=" + fut + ']');
+
+ GridNioWorker worker0 = worker;
+
+ if (worker0 == null) {
+ if (pendingStateChanges == null)
+ pendingStateChanges = new ArrayList<>();
+
+ pendingStateChanges.add(fut);
+ }
+ else
+ worker0.offer(fut);
+ }
+ }
+
+ /**
+ * @param moveFrom Current session worker.
*/
- int selectorIndex() {
- return selectorIdx;
+ void startMoveSession(GridNioWorker moveFrom) {
+ synchronized (this) {
+ assert this.worker == moveFrom;
+
+ if (log.isDebugEnabled())
+ log.debug("Started moving [ses=" + this + ", from=" + moveFrom + ']');
+
+ List<GridNioServer.SessionChangeRequest> sesReqs = moveFrom.clearSessionRequests(this);
+
+ worker = null;
+
+ if (sesReqs != null) {
+ if (pendingStateChanges == null)
+ pendingStateChanges = new ArrayList<>();
+
+ pendingStateChanges.addAll(sesReqs);
+ }
+ }
+ }
+
+ /**
+ * @param moveTo New session worker.
+ */
+ void finishMoveSession(GridNioWorker moveTo) {
+ synchronized (this) {
+ assert worker == null;
+
+ if (log.isDebugEnabled())
+ log.debug("Finishing moving [ses=" + this + ", to=" + moveTo + ']');
+
+ worker = moveTo;
+
+ if (pendingStateChanges != null) {
+ moveTo.offer(pendingStateChanges);
+
+ pendingStateChanges = null;
+ }
+ }
}
/**
@@ -163,14 +259,14 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
* @param writeFut Write request.
* @return Updated size of the queue.
*/
- int offerSystemFuture(GridNioFuture<?> writeFut) {
+ int offerSystemFuture(SessionWriteRequest writeFut) {
writeFut.messageThread(true);
boolean res = queue.offerFirst(writeFut);
assert res : "Future was not added to queue";
- return queueSize.incrementAndGet();
+ return queue.sizex();
}
/**
@@ -183,7 +279,7 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
* @param writeFut Write request to add.
* @return Updated size of the queue.
*/
- int offerFuture(GridNioFuture<?> writeFut) {
+ int offerFuture(SessionWriteRequest writeFut) {
boolean msgThread = GridNioBackPressureControl.threadProcessingMessage();
if (sem != null && !msgThread)
@@ -195,47 +291,41 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
assert res : "Future was not added to queue";
- return queueSize.incrementAndGet();
+ return queue.sizex();
}
/**
* @param futs Futures to resend.
*/
- void resend(Collection<GridNioFuture<?>> futs) {
+ void resend(Collection<SessionWriteRequest> futs) {
assert queue.isEmpty() : queue.size();
boolean add = queue.addAll(futs);
assert add;
-
- boolean set = queueSize.compareAndSet(0, futs.size());
-
- assert set;
}
/**
* @return Message that is in the head of the queue, {@code null} if queue is empty.
*/
- @Nullable GridNioFuture<?> pollFuture() {
- GridNioFuture<?> last = queue.poll();
+ @Nullable SessionWriteRequest pollFuture() {
+ SessionWriteRequest last = queue.poll();
if (last != null) {
- queueSize.decrementAndGet();
-
if (sem != null && !last.messageThread())
sem.release();
- if (recovery != null) {
- if (!recovery.add(last)) {
+ if (outRecovery != null) {
+ if (!outRecovery.add(last)) {
LT.warn(log, "Unacknowledged messages queue size overflow, will attempt to reconnect " +
"[remoteAddr=" + remoteAddress() +
- ", queueLimit=" + recovery.queueLimit() + ']');
+ ", queueLimit=" + outRecovery.queueLimit() + ']');
if (log.isDebugEnabled())
log.debug("Unacknowledged messages queue size overflow, will attempt to reconnect " +
"[remoteAddr=" + remoteAddress() +
- ", queueSize=" + recovery.messagesFutures().size() +
- ", queueLimit=" + recovery.queueLimit() + ']');
+ ", queueSize=" + outRecovery.messagesRequests().size() +
+ ", queueLimit=" + outRecovery.queueLimit() + ']');
close();
}
@@ -249,7 +339,7 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
* @param fut Future.
* @return {@code True} if future was removed from queue.
*/
- boolean removeFuture(GridNioFuture<?> fut) {
+ boolean removeFuture(SessionWriteRequest fut) {
assert closed();
return queue.removeLastOccurrence(fut);
@@ -261,35 +351,49 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
* @return Number of write requests.
*/
int writeQueueSize() {
- return queueSize.get();
+ return queue.sizex();
}
/**
* @return Write requests.
*/
- Collection<GridNioFuture<?>> writeQueue() {
+ Collection<SessionWriteRequest> writeQueue() {
return queue;
}
/** {@inheritDoc} */
- @Override public void recoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+ @Override public void outRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
assert recoveryDesc != null;
- recovery = recoveryDesc;
+ outRecovery = recoveryDesc;
}
/** {@inheritDoc} */
- @Nullable @Override public GridNioRecoveryDescriptor recoveryDescriptor() {
- return recovery;
+ @Nullable @Override public GridNioRecoveryDescriptor outRecoveryDescriptor() {
+ return outRecovery;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void inRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+ assert recoveryDesc != null;
+
+ inRecovery = recoveryDesc;
+ }
+
+ /** {@inheritDoc} */
+ @Nullable @Override public GridNioRecoveryDescriptor inRecoveryDescriptor() {
+ return inRecovery;
}
/** {@inheritDoc} */
@Override public <T> T addMeta(int key, @Nullable T val) {
- if (val instanceof GridNioRecoveryDescriptor) {
- recovery = (GridNioRecoveryDescriptor)val;
+ if (!accepted() && val instanceof GridNioRecoveryDescriptor) {
+ outRecovery = (GridNioRecoveryDescriptor)val;
+
+ if (!outRecovery.pairedConnections())
+ inRecovery = outRecovery;
- if (!accepted())
- recovery.connected();
+ outRecovery.onConnected();
return null;
}
@@ -313,6 +417,31 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
}
/** {@inheritDoc} */
+ @Override public void systemMessage(Object sysMsg) {
+ this.sysMsg = sysMsg;
+ }
+
+ /**
+ * @return {@code True} if have pending system message to send.
+ */
+ boolean hasSystemMessage() {
+ return sysMsg != null;
+ }
+
+ /**
+ * Gets and clears pending system message.
+ *
+ * @return Pending system message.
+ */
+ Object systemMessage() {
+ Object ret = sysMsg;
+
+ sysMsg = null;
+
+ return ret;
+ }
+
+ /** {@inheritDoc} */
@Override public String toString() {
return S.toString(GridSelectorNioSessionImpl.class, this, super.toString());
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
index ebe86fb..d941bae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
@@ -48,6 +48,7 @@ public class GridShmemCommunicationClient extends GridAbstractCommunicationClien
private final MessageFormatter formatter;
/**
+ * @param connIdx Connection index.
* @param metricsLsnr Metrics listener.
* @param port Shared memory IPC server port.
* @param connTimeout Connection timeout.
@@ -55,14 +56,16 @@ public class GridShmemCommunicationClient extends GridAbstractCommunicationClien
* @param formatter Message formatter.
* @throws IgniteCheckedException If failed.
*/
- public GridShmemCommunicationClient(GridNioMetricsListener metricsLsnr,
+ public GridShmemCommunicationClient(
+ int connIdx,
+ GridNioMetricsListener metricsLsnr,
int port,
long connTimeout,
IgniteLogger log,
MessageFormatter formatter)
throws IgniteCheckedException
{
- super(metricsLsnr);
+ super(connIdx, metricsLsnr);
assert metricsLsnr != null;
assert port > 0 && port < 0xffff;
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
index 5fe521d..3397772 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
@@ -45,11 +45,16 @@ public class GridTcpNioCommunicationClient extends GridAbstractCommunicationClie
private final IgniteLogger log;
/**
+ * @param connIdx Connection index.
* @param ses Session.
* @param log Logger.
*/
- public GridTcpNioCommunicationClient(GridNioSession ses, IgniteLogger log) {
- super(null);
+ public GridTcpNioCommunicationClient(
+ int connIdx,
+ GridNioSession ses,
+ IgniteLogger log
+ ) {
+ super(connIdx, null);
assert ses != null;
assert log != null;
@@ -104,40 +109,36 @@ public class GridTcpNioCommunicationClient extends GridAbstractCommunicationClie
}
/** {@inheritDoc} */
- @Override public boolean sendMessage(@Nullable UUID nodeId, Message msg, IgniteInClosure<IgniteException> closure)
+ @Override public boolean sendMessage(@Nullable UUID nodeId, Message msg, IgniteInClosure<IgniteException> c)
throws IgniteCheckedException {
- // Node ID is never provided in asynchronous send mode.
- assert nodeId == null;
+ try {
+ // Node ID is never provided in asynchronous send mode.
+ assert nodeId == null;
- if (closure != null)
- ses.addMeta(ACK_CLOSURE.ordinal(), closure);
+ if (c != null)
+ ses.addMeta(ACK_CLOSURE.ordinal(), c);
- GridNioFuture<?> fut = ses.send(msg);
+ ses.sendNoFuture(msg);
- if (fut.isDone()) {
- try {
- fut.get();
- }
- catch (IgniteCheckedException e) {
- if (closure != null)
- ses.removeMeta(ACK_CLOSURE.ordinal());
+ if (c != null)
+ ses.removeMeta(ACK_CLOSURE.ordinal());
+ }
+ catch (IgniteCheckedException e) {
+ if (c != null)
+ ses.removeMeta(ACK_CLOSURE.ordinal());
- if (log.isDebugEnabled())
- log.debug("Failed to send message [client=" + this + ", err=" + e + ']');
+ if (log.isDebugEnabled())
+ log.debug("Failed to send message [client=" + this + ", err=" + e + ']');
- if (e.getCause() instanceof IOException) {
- ses.close();
+ if (e.getCause() instanceof IOException) {
+ ses.close();
- return true;
- }
- else
- throw new IgniteCheckedException("Failed to send message [client=" + this + ']', e);
+ return true;
}
+ else
+ throw new IgniteCheckedException("Failed to send message [client=" + this + ']', e);
}
- if (closure != null)
- ses.removeMeta(ACK_CLOSURE.ordinal());
-
return false;
}
@@ -159,4 +160,4 @@ public class GridTcpNioCommunicationClient extends GridAbstractCommunicationClie
@Override public String toString() {
return S.toString(GridTcpNioCommunicationClient.class, this, super.toString());
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java
new file mode 100644
index 0000000..508c791
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java
@@ -0,0 +1,85 @@
+/*
+ * 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.util.nio;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.lang.IgniteInClosure;
+
+/**
+ *
+ */
+public interface SessionWriteRequest {
+ /**
+ * Sets flag indicating that message send future was created in thread that was processing a message.
+ *
+ * @param msgThread {@code True} if future was created in thread that is processing message.
+ */
+ public void messageThread(boolean msgThread);
+
+ /**
+ * @return {@code True} if future was created in thread that was processing message.
+ */
+ public boolean messageThread();
+
+ /**
+ * @return {@code True} if skip recovery for this operation.
+ */
+ public boolean skipRecovery();
+
+ /**
+ * Sets ack closure which will be applied when ack received.
+ *
+ * @param c Ack closure.
+ */
+ public void ackClosure(IgniteInClosure<IgniteException> c);
+
+ /**
+ * The method will be called when ack received.
+ */
+ public void onAckReceived();
+
+ /**
+ * @return Ack closure.
+ */
+ public IgniteInClosure<IgniteException> ackClosure();
+
+ /**
+ * @return Session.
+ */
+ public GridNioSession session();
+
+ /**
+ * @param ses Session.
+ */
+ public void resetSession(GridNioSession ses);
+
+ /**
+ *
+ */
+ public void onError(Exception e);
+
+ /**
+ * @return Message.
+ */
+ public Object message();
+
+ /**
+ *
+ */
+ public void onMessageWritten();
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java
index d6f9d10..8ed7db0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java
@@ -282,9 +282,13 @@ public class GridNioSslFilter extends GridNioFilterAdapter {
}
/** {@inheritDoc} */
- @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
+ @Override public GridNioFuture<?> onSessionWrite(
+ GridNioSession ses,
+ Object msg,
+ boolean fut
+ ) throws IgniteCheckedException {
if (directMode)
- return proceedSessionWrite(ses, msg);
+ return proceedSessionWrite(ses, msg, fut);
ByteBuffer input = checkMessage(ses, msg);
@@ -441,4 +445,4 @@ public class GridNioSslFilter extends GridNioFilterAdapter {
return (ByteBuffer)msg;
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java
index eb8dad4..269e8b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java
@@ -437,7 +437,7 @@ class GridNioSslHandler extends ReentrantLock {
while (!deferredWriteQueue.isEmpty()) {
WriteRequest req = deferredWriteQueue.poll();
- req.future().onDone((GridNioFuture<Object>)parent.proceedSessionWrite(ses, req.buffer()));
+ req.future().onDone((GridNioFuture<Object>)parent.proceedSessionWrite(ses, req.buffer(), true));
}
}
@@ -482,7 +482,7 @@ class GridNioSslHandler extends ReentrantLock {
ByteBuffer cp = copy(outNetBuf);
- return parent.proceedSessionWrite(ses, cp);
+ return parent.proceedSessionWrite(ses, cp, true);
}
/**
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
index b29d7cd..86aa7a1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
@@ -598,4 +598,4 @@ public class GridToStringBuilder {
return cd;
}
-}
\ No newline at end of file
+}
[15/25] ignite git commit: IGNITE-3220 I/O bottleneck on
server/client cluster configuration Communications optimizations: -
possibility to open separate in/out connections - possibility to have
multiple connections between nodes - implemented NIO sessio
Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
index b0353a6..5345a9b 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
@@ -18,8 +18,6 @@
package org.apache.ignite.spi.communication.tcp;
import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
import org.apache.ignite.testframework.junits.spi.GridSpiAbstractConfigTest;
import org.apache.ignite.testframework.junits.spi.GridSpiTest;
@@ -47,6 +45,9 @@ public class GridTcpCommunicationSpiConfigSelfTest extends GridSpiAbstractConfig
checkNegativeSpiProperty(new TcpCommunicationSpi(), "ackSendThreshold", 0);
checkNegativeSpiProperty(new TcpCommunicationSpi(), "ackSendThreshold", -1);
checkNegativeSpiProperty(new TcpCommunicationSpi(), "unacknowledgedMessagesBufferSize", -1);
+ checkNegativeSpiProperty(new TcpCommunicationSpi(), "connectionsPerNode", 0);
+ checkNegativeSpiProperty(new TcpCommunicationSpi(), "connectionsPerNode", -1);
+ checkNegativeSpiProperty(new TcpCommunicationSpi(), "connectionsPerNode", Integer.MAX_VALUE);
}
/**
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
index a74a07a..0c57105 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
@@ -109,7 +109,7 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
/**
* @param useShmem Use shared mem.
*/
- protected GridTcpCommunicationSpiMultithreadedSelfTest(boolean useShmem) {
+ GridTcpCommunicationSpiMultithreadedSelfTest(boolean useShmem) {
super(false);
this.useShmem = useShmem;
@@ -370,17 +370,17 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
Collection<? extends GridNioSession> sessions = GridTestUtils.getFieldValue(srv, "sessions");
for (GridNioSession ses : sessions) {
- final GridNioRecoveryDescriptor snd = ses.recoveryDescriptor();
+ final GridNioRecoveryDescriptor snd = ses.outRecoveryDescriptor();
if (snd != null) {
GridTestUtils.waitForCondition(new GridAbsPredicate() {
@Override public boolean apply() {
- return snd.messagesFutures().isEmpty();
+ return snd.messagesRequests().isEmpty();
}
}, 10_000);
- assertEquals("Unexpected messages: " + snd.messagesFutures(), 0,
- snd.messagesFutures().size());
+ assertEquals("Unexpected messages: " + snd.messagesRequests(), 0,
+ snd.messagesRequests().size());
}
}
}
@@ -547,11 +547,18 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
}
for (CommunicationSpi spi : spis.values()) {
- final ConcurrentMap<UUID, GridCommunicationClient> clients = U.field(spi, "clients");
+ final ConcurrentMap<UUID, GridCommunicationClient[]> clients = U.field(spi, "clients");
assert GridTestUtils.waitForCondition(new PA() {
@Override public boolean apply() {
- return clients.isEmpty();
+ for (GridCommunicationClient[] clients0 : clients.values()) {
+ for (GridCommunicationClient client : clients0) {
+ if (client != null)
+ return false;
+ }
+ }
+
+ return true;
}
}, getTestTimeout()) : "Clients: " + clients;
}
@@ -583,4 +590,4 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
spis.clear();
nodes.clear();
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
index 34872c6..12c2edb 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
@@ -173,7 +173,7 @@ public class GridTcpCommunicationSpiRecoveryAckSelfTest<T extends CommunicationS
boolean found = false;
for (GridNioSession ses : sessions) {
- final GridNioRecoveryDescriptor recoveryDesc = ses.recoveryDescriptor();
+ final GridNioRecoveryDescriptor recoveryDesc = ses.outRecoveryDescriptor();
if (recoveryDesc != null) {
found = true;
@@ -188,12 +188,12 @@ public class GridTcpCommunicationSpiRecoveryAckSelfTest<T extends CommunicationS
GridTestUtils.waitForCondition(new GridAbsPredicate() {
@Override public boolean apply() {
- return recoveryDesc.messagesFutures().isEmpty();
+ return recoveryDesc.messagesRequests().isEmpty();
}
}, 10_000);
- assertEquals("Unexpected messages: " + recoveryDesc.messagesFutures(), 0,
- recoveryDesc.messagesFutures().size());
+ assertEquals("Unexpected messages: " + recoveryDesc.messagesRequests(), 0,
+ recoveryDesc.messagesRequests().size());
break;
}
@@ -361,6 +361,7 @@ public class GridTcpCommunicationSpiRecoveryAckSelfTest<T extends CommunicationS
spi.setAckSendThreshold(ackCnt);
spi.setMessageQueueLimit(queueLimit);
spi.setSharedMemoryPort(-1);
+ spi.setConnectionsPerNode(1);
return spi;
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest.java
index 95c9e40..b1aa119 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest.java
@@ -33,6 +33,7 @@ public class GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest extends Gri
spi.setAckSendThreshold(5);
spi.setSocketSendBuffer(512);
spi.setSocketReceiveBuffer(512);
+ spi.setConnectionsPerNode(1);
return spi;
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest.java
new file mode 100644
index 0000000..8e43937
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spi.communication.tcp;
+
+/**
+ *
+ */
+public class GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest extends GridTcpCommunicationSpiRecoverySelfTest {
+ /** {@inheritDoc} */
+ @Override protected boolean usePairedConnections() {
+ return false;
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
index 3234d74..065a3d7 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
@@ -56,6 +56,7 @@ import org.eclipse.jetty.util.ConcurrentHashSet;
/**
*
*/
+@SuppressWarnings("unchecked")
@GridSpiTest(spi = TcpCommunicationSpi.class, group = "Communication SPI")
public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi> extends GridSpiAbstractTest<T> {
/** */
@@ -305,14 +306,14 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
log.info("Iteration: " + i);
try {
- final GridNioSession ses0 = communicationSession(spi0);
- final GridNioSession ses1 = communicationSession(spi1);
+ final GridNioSession ses0 = communicationSession(spi0, false);
+ final GridNioSession ses1 = communicationSession(spi1, true);
ses1.pauseReads().get();
IgniteInternalFuture<?> sndFut = GridTestUtils.runAsync(new Callable<Void>() {
@Override public Void call() throws Exception {
- for (int i = 0; i < 5000; i++) {
+ for (int i = 0; i < 6000; i++) {
spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
sentCnt.incrementAndGet();
@@ -331,7 +332,12 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
assertTrue("Failed to wait for session close", ses0.closeTime() != 0);
- ses1.resumeReads().get();
+ try {
+ ses1.resumeReads().get();
+ }
+ catch (IgniteCheckedException ignore) {
+ // Can fail is ses1 was closed.
+ }
for (int j = 0; j < 100; j++) {
spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
@@ -415,14 +421,14 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
log.info("Iteration: " + i);
try {
- final GridNioSession ses0 = communicationSession(spi0);
- final GridNioSession ses1 = communicationSession(spi1);
+ final GridNioSession ses0 = communicationSession(spi0, false);
+ final GridNioSession ses1 = communicationSession(spi1, true);
ses1.pauseReads().get();
IgniteInternalFuture<?> sndFut = GridTestUtils.runAsync(new Callable<Void>() {
@Override public Void call() throws Exception {
- for (int i = 0; i < 5000; i++) {
+ for (int i = 0; i < 6000; i++) {
spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
expCnt1.incrementAndGet();
@@ -441,12 +447,16 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
assertTrue("Failed to wait for session close", ses0.closeTime() != 0);
- ses1.resumeReads().get();
+ try {
+ ses1.resumeReads().get();
+ }
+ catch (IgniteCheckedException ignore) {
+ // Can fail is ses1 was closed.
+ }
// Wait when session is closed, then try to open new connection from node1.
GridTestUtils.waitForCondition(new GridAbsPredicate() {
- @Override
- public boolean apply() {
+ @Override public boolean apply() {
return ses1.closeTime() != 0;
}
}, awaitForSocketWriteTimeout());
@@ -532,14 +542,14 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
log.info("Iteration: " + i);
try {
- final GridNioSession ses0 = communicationSession(spi0);
- final GridNioSession ses1 = communicationSession(spi1);
+ final GridNioSession ses0 = communicationSession(spi0, false);
+ final GridNioSession ses1 = communicationSession(spi1, true);
ses1.pauseReads().get();
IgniteInternalFuture<?> sndFut = GridTestUtils.runAsync(new Callable<Void>() {
@Override public Void call() throws Exception {
- for (int i = 0; i < 5000; i++) {
+ for (int i = 0; i < 6000; i++) {
spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
sentCnt.incrementAndGet();
@@ -558,7 +568,12 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
assertTrue("Failed to wait for session close", ses0.closeTime() != 0);
- ses1.resumeReads().get();
+ try {
+ ses1.resumeReads().get();
+ }
+ catch (IgniteCheckedException ignore) {
+ // Can fail is ses1 was closed.
+ }
sndFut.get();
@@ -605,11 +620,12 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
/**
* @param spi SPI.
+ * @param in {@code True} if need find inbound session.
* @return Session.
* @throws Exception If failed.
*/
@SuppressWarnings("unchecked")
- private GridNioSession communicationSession(TcpCommunicationSpi spi) throws Exception {
+ private GridNioSession communicationSession(TcpCommunicationSpi spi, boolean in) throws Exception {
final GridNioServer srv = U.field(spi, "nioSrvr");
GridTestUtils.waitForCondition(new GridAbsPredicate() {
@@ -622,9 +638,21 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
Collection<? extends GridNioSession> sessions = GridTestUtils.getFieldValue(srv, "sessions");
- assertEquals(1, sessions.size());
+ for (GridNioSession ses : sessions) {
+ if (in == ses.accepted())
+ return ses;
+ }
+
+ fail("Failed to find session");
- return sessions.iterator().next();
+ return null;
+ }
+
+ /**
+ * @return {@code True}.
+ */
+ protected boolean usePairedConnections() {
+ return true;
}
/**
@@ -642,6 +670,8 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
spi.setSocketWriteTimeout(1000);
spi.setSocketSendBuffer(512);
spi.setSocketReceiveBuffer(512);
+ spi.setConnectionsPerNode(1);
+ spi.setUsePairedConnections(usePairedConnections());
return spi;
}
@@ -748,9 +778,8 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
spi.spiStop();
}
- for (IgniteTestResources rsrcs : spiRsrcs) {
+ for (IgniteTestResources rsrcs : spiRsrcs)
rsrcs.stopThreads();
- }
spis.clear();
nodes.clear();
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java
index 25e3611..c4930a0 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java
@@ -187,7 +187,7 @@ public class IgniteTcpCommunicationRecoveryAckClosureSelfTest<T extends Communic
boolean found = false;
for (GridNioSession ses : sessions) {
- final GridNioRecoveryDescriptor recoveryDesc = ses.recoveryDescriptor();
+ final GridNioRecoveryDescriptor recoveryDesc = ses.outRecoveryDescriptor();
if (recoveryDesc != null) {
found = true;
@@ -202,12 +202,12 @@ public class IgniteTcpCommunicationRecoveryAckClosureSelfTest<T extends Communic
GridTestUtils.waitForCondition(new GridAbsPredicate() {
@Override public boolean apply() {
- return recoveryDesc.messagesFutures().isEmpty();
+ return recoveryDesc.messagesRequests().isEmpty();
}
}, 10_000);
- assertEquals("Unexpected messages: " + recoveryDesc.messagesFutures(), 0,
- recoveryDesc.messagesFutures().size());
+ assertEquals("Unexpected messages: " + recoveryDesc.messagesRequests(), 0,
+ recoveryDesc.messagesRequests().size());
break;
}
@@ -397,6 +397,7 @@ public class IgniteTcpCommunicationRecoveryAckClosureSelfTest<T extends Communic
spi.setAckSendThreshold(ackCnt);
spi.setMessageQueueLimit(queueLimit);
spi.setSharedMemoryPort(-1);
+ spi.setConnectionsPerNode(1);
return spi;
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
index f9e2ff4..143159d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
@@ -63,7 +63,9 @@ public class GridTestKernalContext extends GridKernalContextImpl {
null,
null,
null,
- U.allPluginProviders());
+ null,
+ U.allPluginProviders()
+ );
GridTestUtils.setFieldValue(grid(), "cfg", config());
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
index 14e5833..17757ab 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
@@ -57,6 +57,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAto
import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderFairAffinityMultiNodeFullApiSelfTest;
import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderMultiNodeFullApiSelfTest;
import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderMultiNodeP2PDisabledFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest;
import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderOffHeapFullApiSelfTest;
import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderOffHeapTieredFullApiSelfTest;
import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWrityOrderOffHeapMultiNodeFullApiSelfTest;
@@ -77,6 +78,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePar
import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeP2PDisabledFullApiSelfTest;
import org.apache.ignite.internal.processors.cache.distributed.near.CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest;
import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedNearOnlyNoPrimaryFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest;
import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapFullApiSelfTest;
import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapMultiNodeFullApiSelfTest;
import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapTieredFullApiSelfTest;
@@ -228,6 +230,10 @@ public class IgniteCacheFullApiSelfTestSuite extends TestSuite {
suite.addTestSuite(GridCacheReplicatedFullApiMultithreadedSelfTest.class);
suite.addTestSuite(GridCachePartitionedFullApiMultithreadedSelfTest.class);
+ // Disabled striped pool.
+ suite.addTestSuite(GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest.class);
+ suite.addTestSuite(GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest.class);
+
// Other.
suite.addTestSuite(GridCacheClearSelfTest.class);
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 435fcfb..554bb3d 100755
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -39,6 +39,10 @@ import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreOptimizedMarshallerW
import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreTest;
import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreMultithreadedSelfTest;
import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreSelfTest;
+import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceMultipleConnectionsTest;
+import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceTest;
+import org.apache.ignite.internal.managers.communication.IgniteIoTestMessagesTest;
+import org.apache.ignite.internal.managers.communication.IgniteVariousConnectionNumberTest;
import org.apache.ignite.internal.processors.cache.CacheAffinityCallSelfTest;
import org.apache.ignite.internal.processors.cache.CacheDeferredDeleteSanitySelfTest;
import org.apache.ignite.internal.processors.cache.CacheEntryProcessorCopySelfTest;
@@ -128,7 +132,10 @@ import org.apache.ignite.internal.processors.cache.context.IgniteCacheTxExecutio
import org.apache.ignite.internal.processors.cache.distributed.CacheAtomicNearUpdateTopologyChangeTest;
import org.apache.ignite.internal.processors.cache.distributed.CacheTxNearUpdateTopologyChangeTest;
import org.apache.ignite.internal.processors.cache.distributed.GridCacheEntrySetIterationPreloadingSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheAtomicMessageRecovery10ConnectionsTest;
+import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest;
import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheAtomicMessageRecoveryTest;
+import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheConnectionRecovery10ConnectionsTest;
import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheConnectionRecoveryTest;
import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheMessageRecoveryIdleConnectionTest;
import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheMessageWriteTimeoutTest;
@@ -259,7 +266,7 @@ public class IgniteCacheTestSuite extends TestSuite {
suite.addTestSuite(GridCacheStoreValueBytesSelfTest.class);
GridTestUtils.addTestIfNeeded(suite, DataStreamProcessorSelfTest.class, ignoredTests);
GridTestUtils.addTestIfNeeded(suite, DataStreamerUpdateAfterLoadTest.class, ignoredTests);
- suite.addTestSuite(DataStreamerMultiThreadedSelfTest.class);
+ suite.addTestSuite(DataStreamerMultiThreadedSelfTest.class);
suite.addTestSuite(DataStreamerMultinodeCreateCacheTest.class);
suite.addTestSuite(DataStreamerImplSelfTest.class);
suite.addTestSuite(DataStreamerTimeoutTest.class);
@@ -292,10 +299,13 @@ public class IgniteCacheTestSuite extends TestSuite {
suite.addTestSuite(GridCacheEntrySetIterationPreloadingSelfTest.class);
suite.addTestSuite(GridCacheMixedPartitionExchangeSelfTest.class);
suite.addTestSuite(IgniteCacheAtomicMessageRecoveryTest.class);
+ suite.addTestSuite(IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest.class);
+ suite.addTestSuite(IgniteCacheAtomicMessageRecovery10ConnectionsTest.class);
suite.addTestSuite(IgniteCacheTxMessageRecoveryTest.class);
suite.addTestSuite(IgniteCacheMessageWriteTimeoutTest.class);
suite.addTestSuite(IgniteCacheMessageRecoveryIdleConnectionTest.class);
suite.addTestSuite(IgniteCacheConnectionRecoveryTest.class);
+ suite.addTestSuite(IgniteCacheConnectionRecovery10ConnectionsTest.class);
GridTestUtils.addTestIfNeeded(suite, GridCacheOffHeapTieredEvictionAtomicSelfTest.class, ignoredTests);
GridTestUtils.addTestIfNeeded(suite, GridCacheOffHeapTieredEvictionSelfTest.class, ignoredTests);
GridTestUtils.addTestIfNeeded(suite, GridCacheOffHeapTieredAtomicSelfTest.class, ignoredTests);
@@ -325,6 +335,11 @@ public class IgniteCacheTestSuite extends TestSuite {
suite.addTestSuite(CacheTxFastFinishTest.class);
+ suite.addTestSuite(IgniteVariousConnectionNumberTest.class);
+ suite.addTestSuite(IgniteCommunicationBalanceTest.class);
+ suite.addTestSuite(IgniteCommunicationBalanceMultipleConnectionsTest.class);
+ suite.addTestSuite(IgniteIoTestMessagesTest.class);
+
return suite;
}
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java
index c557fbb..11fcfda 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java
@@ -25,6 +25,7 @@ import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiMultithrea
import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiMultithreadedShmemTest;
import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoveryAckSelfTest;
import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest;
+import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest;
import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoverySelfTest;
import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoverySslSelfTest;
import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiShmemSelfTest;
@@ -50,6 +51,7 @@ public class IgniteSpiCommunicationSelfTestSuite extends TestSuite {
suite.addTest(new TestSuite(GridTcpCommunicationSpiRecoveryAckSelfTest.class));
suite.addTest(new TestSuite(IgniteTcpCommunicationRecoveryAckClosureSelfTest.class));
suite.addTest(new TestSuite(GridTcpCommunicationSpiRecoverySelfTest.class));
+ suite.addTest(new TestSuite(GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest.class));
suite.addTest(new TestSuite(GridTcpCommunicationSpiRecoverySslSelfTest.class));
suite.addTest(new TestSuite(GridTcpCommunicationSpiConcurrentConnectSelfTest.class));
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
index a725534..9542372 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
@@ -240,7 +240,7 @@ public class HadoopJobTracker extends HadoopComponent {
try {
// Must process query callback in a separate thread to avoid deadlocks.
- evtProcSvc.submit(new EventHandler() {
+ evtProcSvc.execute(new EventHandler() {
@Override protected void body() throws IgniteCheckedException {
processJobMetadataUpdates(evts);
}
@@ -264,7 +264,7 @@ public class HadoopJobTracker extends HadoopComponent {
try {
// Must process discovery callback in a separate thread to avoid deadlock.
- evtProcSvc.submit(new EventHandler() {
+ evtProcSvc.execute(new EventHandler() {
@Override protected void body() {
processNodeLeft((DiscoveryEvent)evt);
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
index bc047e7..ff58509 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
@@ -639,6 +639,7 @@ public class HadoopExternalCommunication {
.logger(log.getLogger(GridNioServer.class))
.selectorCount(selectorsCnt)
.gridName(gridName)
+ .serverName("hadoop")
.tcpNoDelay(tcpNoDelay)
.directBuffer(directBuf)
.byteOrder(ByteOrder.nativeOrder())
@@ -1305,11 +1306,11 @@ public class HadoopExternalCommunication {
}
/** {@inheritDoc} */
- @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
+ @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) throws IgniteCheckedException {
if (ses.meta(PROCESS_META) == null && !(msg instanceof ProcessHandshakeMessage))
log.warning("Writing message before handshake has finished [ses=" + ses + ", msg=" + msg + ']');
- return proceedSessionWrite(ses, msg);
+ return proceedSessionWrite(ses, msg, fut);
}
/** {@inheritDoc} */
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java
index a8de999..3f33fb7 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java
@@ -190,7 +190,7 @@ public class HadoopIpcToNioAdapter<T> {
}
/** {@inheritDoc} */
- @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) {
+ @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) {
assert ses == HadoopIpcToNioAdapter.this.ses : "ses=" + ses +
", this.ses=" + HadoopIpcToNioAdapter.this.ses;
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java
index eeca564..24bba88 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java
@@ -57,10 +57,10 @@ public class HadoopMarshallerFilter extends GridNioFilterAdapter {
}
/** {@inheritDoc} */
- @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
+ @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) throws IgniteCheckedException {
assert msg instanceof HadoopMessage : "Invalid message type: " + msg;
- return proceedSessionWrite(ses, U.marshal(marsh, msg));
+ return proceedSessionWrite(ses, U.marshal(marsh, msg), fut);
}
@Override public void onMessageReceived(GridNioSession ses, Object msg) throws IgniteCheckedException {
@@ -84,4 +84,4 @@ public class HadoopMarshallerFilter extends GridNioFilterAdapter {
@Override public void onSessionWriteTimeout(GridNioSession ses) throws IgniteCheckedException {
proceedSessionWriteTimeout(ses);
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/KafkaStreamer.java
----------------------------------------------------------------------
diff --git a/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/KafkaStreamer.java b/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/KafkaStreamer.java
index 5767790..73820b7 100644
--- a/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/KafkaStreamer.java
+++ b/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/KafkaStreamer.java
@@ -143,7 +143,7 @@ public class KafkaStreamer<K, V> extends StreamAdapter<MessageAndMetadata<byte[]
// Now create an object to consume the messages.
for (final KafkaStream<byte[], byte[]> stream : streams) {
- executor.submit(new Runnable() {
+ executor.execute(new Runnable() {
@Override public void run() {
while (!stopped) {
try {
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/tools/src/main/java/org/apache/ignite/tools/classgen/ClassesGenerator.java
----------------------------------------------------------------------
diff --git a/modules/tools/src/main/java/org/apache/ignite/tools/classgen/ClassesGenerator.java b/modules/tools/src/main/java/org/apache/ignite/tools/classgen/ClassesGenerator.java
index 0b10a75..369e318 100644
--- a/modules/tools/src/main/java/org/apache/ignite/tools/classgen/ClassesGenerator.java
+++ b/modules/tools/src/main/java/org/apache/ignite/tools/classgen/ClassesGenerator.java
@@ -122,7 +122,11 @@ public class ClassesGenerator {
for (String err : errs)
sb.append(" ").append(err).append('\n');
- throw new Exception(sb.toString().trim());
+ String msg = sb.toString().trim();
+
+ System.out.println(msg);
+
+ throw new Exception(msg);
}
PrintStream out = new PrintStream(new File(basePath,
@@ -246,4 +250,4 @@ public class ClassesGenerator {
}
}
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkUtils.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkUtils.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkUtils.java
index 54ef00d..07549d5 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkUtils.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkUtils.java
@@ -30,7 +30,7 @@ import org.apache.ignite.transactions.TransactionConcurrency;
import org.apache.ignite.transactions.TransactionIsolation;
import org.apache.ignite.transactions.TransactionOptimisticException;
import org.apache.ignite.transactions.TransactionRollbackException;
-import org.apache.ignite.yardstick.cache.IgniteSqlQueryBenchmark;
+import org.apache.ignite.yardstick.cache.IgnitePutBenchmark;
import org.yardstickframework.BenchmarkDriver;
import org.yardstickframework.BenchmarkDriverStartUp;
@@ -89,13 +89,13 @@ public class IgniteBenchmarkUtils {
public static void main(String[] args) throws Exception {
final String cfg = "modules/yardstick/config/ignite-localhost-config.xml";
- final Class<? extends BenchmarkDriver> benchmark = IgniteSqlQueryBenchmark.class;
+ final Class<? extends BenchmarkDriver> benchmark = IgnitePutBenchmark.class;
final int threads = 1;
final boolean clientDriverNode = true;
- final int extraNodes = 2;
+ final int extraNodes = 4;
final int warmUp = 5;
final int duration = 5;
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/CacheEntryEventProbe.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/CacheEntryEventProbe.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/CacheEntryEventProbe.java
index a25f975..c8022e2 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/CacheEntryEventProbe.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/CacheEntryEventProbe.java
@@ -72,7 +72,7 @@ public class CacheEntryEventProbe implements BenchmarkProbe {
buildingService = Executors.newSingleThreadExecutor();
- buildingService.submit(new Runnable() {
+ buildingService.execute(new Runnable() {
@Override public void run() {
try {
while (!Thread.currentThread().isInterrupted()) {
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteIoTestBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteIoTestBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteIoTestBenchmark.java
new file mode 100644
index 0000000..bee45e0
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteIoTestBenchmark.java
@@ -0,0 +1,73 @@
+/*
+ * 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.yardstick.cache;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.yardstick.IgniteAbstractBenchmark;
+import org.yardstickframework.BenchmarkConfiguration;
+import org.yardstickframework.BenchmarkUtils;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+/**
+ *
+ */
+public class IgniteIoTestBenchmark extends IgniteAbstractBenchmark {
+ /** */
+ private List<ClusterNode> targetNodes;
+
+ /** */
+ private IgniteKernal ignite;
+
+ /** {@inheritDoc} */
+ @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+ super.setUp(cfg);
+
+ ignite = (IgniteKernal)ignite();
+
+ targetNodes = new ArrayList<>();
+
+ ClusterNode loc = ignite().cluster().localNode();
+
+ Collection<ClusterNode> nodes = ignite().cluster().forServers().nodes();
+
+ for (ClusterNode node : nodes) {
+ if (!loc.equals(node))
+ targetNodes.add(node);
+ }
+
+ if (targetNodes.isEmpty())
+ throw new IgniteException("Failed to find remote server nodes [nodes=" + nodes + ']');
+
+ BenchmarkUtils.println(cfg, "Initialized target nodes: " + targetNodes + ']');
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+ ClusterNode node = targetNodes.get(nextRandom(targetNodes.size()));
+
+ ignite.sendIoTest(node, null, false).get();
+
+ return true;
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestAbstractBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestAbstractBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestAbstractBenchmark.java
new file mode 100644
index 0000000..8791c83
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestAbstractBenchmark.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.yardstick.io;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.yardstick.IgniteAbstractBenchmark;
+import org.yardstickframework.BenchmarkConfiguration;
+import org.yardstickframework.BenchmarkUtils;
+
+/**
+ *
+ */
+public abstract class IgniteIoTestAbstractBenchmark extends IgniteAbstractBenchmark {
+ /** */
+ protected final List<ClusterNode> targetNodes = new ArrayList<>();
+
+ /** */
+ protected IgniteKernal ignite;
+
+ /** {@inheritDoc} */
+ @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+ super.setUp(cfg);
+
+ ignite = (IgniteKernal)ignite();
+
+ ClusterNode loc = ignite().cluster().localNode();
+
+ Collection<ClusterNode> nodes = ignite().cluster().forServers().nodes();
+
+ for (ClusterNode node : nodes) {
+ if (!loc.equals(node))
+ targetNodes.add(node);
+ }
+
+ if (targetNodes.isEmpty())
+ throw new IgniteException("Failed to find remote server nodes [nodes=" + nodes + ']');
+
+ BenchmarkUtils.println(cfg, "Initialized target nodes: " + F.nodeIds(targetNodes) + ']');
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendAllBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendAllBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendAllBenchmark.java
new file mode 100644
index 0000000..9011910
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendAllBenchmark.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.yardstick.io;
+
+import java.util.Map;
+
+/**
+ *
+ */
+public class IgniteIoTestSendAllBenchmark extends IgniteIoTestAbstractBenchmark {
+ /** {@inheritDoc} */
+ @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+ ignite.sendIoTest(targetNodes, null, false).get();
+
+ return true;
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendRandomBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendRandomBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendRandomBenchmark.java
new file mode 100644
index 0000000..88368e0
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendRandomBenchmark.java
@@ -0,0 +1,35 @@
+/*
+ * 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.yardstick.io;
+
+import java.util.Map;
+import org.apache.ignite.cluster.ClusterNode;
+
+/**
+ *
+ */
+public class IgniteIoTestSendRandomBenchmark extends IgniteIoTestAbstractBenchmark {
+ /** {@inheritDoc} */
+ @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+ ClusterNode node = targetNodes.get(nextRandom(targetNodes.size()));
+
+ ignite.sendIoTest(node, null, false).get();
+
+ return true;
+ }
+}
[09/25] ignite git commit: IGNITE-4027 Extract PlatformTargetProxy
interface
Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
index 8ff15d5..fd1c2d4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx;
import org.apache.ignite.internal.binary.BinaryRawWriterEx;
import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformTarget;
import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils;
import org.apache.ignite.internal.processors.platform.utils.PlatformListenable;
import org.apache.ignite.internal.util.future.IgniteFutureImpl;
@@ -98,7 +99,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected Object processInStreamOutObject(int type, BinaryRawReaderEx reader)
+ @Override public PlatformTarget processInStreamOutObject(int type, BinaryRawReaderEx reader)
throws IgniteCheckedException {
switch (type) {
case OP_UNICAST:
@@ -120,7 +121,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
}
case OP_EXEC_ASYNC:
- return executeJavaTask(reader, true);
+ return wrapListenable((PlatformListenable) executeJavaTask(reader, true));
default:
return super.processInStreamOutObject(type, reader);
@@ -128,7 +129,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+ @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
switch (type) {
case OP_WITH_TIMEOUT: {
compute.withTimeout(val);
@@ -154,7 +155,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
* @param reader Reader.
* @param broadcast broadcast flag.
*/
- private PlatformListenable processClosures(long taskPtr, BinaryRawReaderEx reader, boolean broadcast,
+ private PlatformTarget processClosures(long taskPtr, BinaryRawReaderEx reader, boolean broadcast,
boolean affinity) {
PlatformAbstractTask task;
@@ -221,7 +222,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
+ @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
throws IgniteCheckedException {
switch (type) {
case OP_EXEC:
@@ -239,7 +240,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
*
* @param task Task.
*/
- private PlatformListenable executeNative0(final PlatformAbstractTask task) {
+ private PlatformTarget executeNative0(final PlatformAbstractTask task) {
IgniteInternalFuture fut = computeForPlatform.executeAsync(task, null);
fut.listen(new IgniteInClosure<IgniteInternalFuture>() {
@@ -257,7 +258,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
}
});
- return PlatformFutureUtils.getListenable(fut);
+ return wrapListenable(PlatformFutureUtils.getListenable(fut));
}
/**
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
index cd5fba0..7d71a9e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
@@ -114,7 +114,7 @@ public class PlatformDataStreamer extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+ @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
switch (type) {
case OP_UPDATE:
int plc = reader.readInt();
@@ -169,7 +169,7 @@ public class PlatformDataStreamer extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInLongOutLong(int type, final long val) throws IgniteCheckedException {
+ @Override public long processInLongOutLong(int type, final long val) throws IgniteCheckedException {
switch (type) {
case OP_SET_ALLOW_OVERWRITE:
ldr.allowOverwrite(val == TRUE);
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiverImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiverImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiverImpl.java
index add11ed..d0992fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiverImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiverImpl.java
@@ -23,6 +23,8 @@ import org.apache.ignite.IgniteException;
import org.apache.ignite.internal.binary.BinaryRawWriterEx;
import org.apache.ignite.internal.processors.platform.PlatformAbstractPredicate;
import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformTargetProxy;
+import org.apache.ignite.internal.processors.platform.PlatformTargetProxyImpl;
import org.apache.ignite.internal.processors.platform.cache.PlatformCache;
import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
@@ -89,8 +91,10 @@ public class PlatformStreamReceiverImpl extends PlatformAbstractPredicate implem
out.synchronize();
- ctx.gateway().dataStreamerStreamReceiverInvoke(ptr, new PlatformCache(ctx, cache, keepBinary),
- mem.pointer(), keepBinary);
+ PlatformCache cache0 = new PlatformCache(ctx, cache, keepBinary);
+ PlatformTargetProxy cacheProxy = new PlatformTargetProxyImpl(cache0, ctx);
+
+ ctx.gateway().dataStreamerStreamReceiverInvoke(ptr, cacheProxy, mem.pointer(), keepBinary);
}
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java
index 811e38b..b57b140 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java
@@ -80,7 +80,7 @@ public class PlatformAtomicLong extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+ @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
switch (type) {
case OP_COMPARE_AND_SET:
long cmp = reader.readLong();
@@ -99,7 +99,7 @@ public class PlatformAtomicLong extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+ @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
switch (type) {
case OP_ADD_AND_GET:
return atomicLong.addAndGet(val);
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java
index 63b5b86..a644259 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java
@@ -94,7 +94,7 @@ public class PlatformAtomicReference extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+ @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
if (type == OP_GET)
writer.writeObject(atomicRef.get());
else
@@ -102,7 +102,7 @@ public class PlatformAtomicReference extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader)
+ @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader)
throws IgniteCheckedException {
if (type == OP_SET) {
atomicRef.set(reader.readObjectDetached());
@@ -114,7 +114,7 @@ public class PlatformAtomicReference extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader,
+ @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader,
BinaryRawWriterEx writer) throws IgniteCheckedException {
if (type == OP_COMPARE_AND_SET_AND_GET) {
Object val = reader.readObjectDetached();
@@ -134,7 +134,7 @@ public class PlatformAtomicReference extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+ @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
switch (type) {
case OP_CLOSE:
atomicRef.close();
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java
index c352731..6d17a72 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java
@@ -71,7 +71,7 @@ public class PlatformAtomicSequence extends PlatformAbstractTarget {
/** {@inheritDoc} */
- @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+ @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
switch (type) {
case OP_ADD_AND_GET:
return atomicSeq.addAndGet(val);
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
index 383e7ab..9ddcc37 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.binary.BinaryRawWriterEx;
import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
import org.apache.ignite.internal.processors.platform.PlatformContext;
import org.apache.ignite.internal.processors.platform.PlatformEventFilterListener;
+import org.apache.ignite.internal.processors.platform.PlatformTarget;
import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils;
import org.apache.ignite.internal.util.future.IgniteFutureImpl;
import org.apache.ignite.internal.util.typedef.F;
@@ -117,7 +118,7 @@ public class PlatformEvents extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader)
+ @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader)
throws IgniteCheckedException {
switch (type) {
case OP_RECORD_LOCAL:
@@ -168,7 +169,7 @@ public class PlatformEvents extends PlatformAbstractTarget {
/** {@inheritDoc} */
@SuppressWarnings({"IfMayBeConditional", "ConstantConditions", "unchecked"})
- @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
+ @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
throws IgniteCheckedException {
switch (type) {
case OP_LOCAL_QUERY: {
@@ -271,7 +272,7 @@ public class PlatformEvents extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+ @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
switch (type) {
case OP_GET_ENABLED_EVENTS:
writeEventTypes(events.enabledEvents(), writer);
@@ -284,7 +285,7 @@ public class PlatformEvents extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected Object processOutObject(int type) throws IgniteCheckedException {
+ @Override public PlatformTarget processOutObject(int type) throws IgniteCheckedException {
switch (type) {
case OP_WITH_ASYNC:
if (events.isAsync())
@@ -297,7 +298,7 @@ public class PlatformEvents extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+ @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
switch (type) {
case OP_IS_ENABLED:
return events.isEnabled((int)val) ? TRUE : FALSE;
@@ -310,12 +311,12 @@ public class PlatformEvents extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected IgniteInternalFuture currentFuture() throws IgniteCheckedException {
+ @Override public IgniteInternalFuture currentFuture() throws IgniteCheckedException {
return ((IgniteFutureImpl)eventsAsync.future()).internalFuture();
}
/** {@inheritDoc} */
- @Nullable @Override protected PlatformFutureUtils.Writer futureWriter(int opId) {
+ @Nullable @Override public PlatformFutureUtils.Writer futureWriter(int opId) {
switch (opId) {
case OP_WAIT_FOR_LOCAL:
return eventResWriter;
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
index 216427a..6fe109e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
@@ -24,6 +24,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx;
import org.apache.ignite.internal.binary.BinaryRawWriterEx;
import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformTarget;
import org.apache.ignite.internal.processors.platform.message.PlatformMessageFilter;
import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
import org.apache.ignite.internal.util.future.IgniteFutureImpl;
@@ -86,7 +87,7 @@ public class PlatformMessaging extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader)
+ @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader)
throws IgniteCheckedException {
switch (type) {
case OP_SEND:
@@ -149,7 +150,7 @@ public class PlatformMessaging extends PlatformAbstractTarget {
/** {@inheritDoc} */
@SuppressWarnings({"IfMayBeConditional", "ConstantConditions", "unchecked"})
- @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
+ @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
throws IgniteCheckedException {
switch (type) {
case OP_REMOTE_LISTEN:{
@@ -181,12 +182,12 @@ public class PlatformMessaging extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected IgniteInternalFuture currentFuture() throws IgniteCheckedException {
+ @Override public IgniteInternalFuture currentFuture() throws IgniteCheckedException {
return ((IgniteFutureImpl)messagingAsync.future()).internalFuture();
}
/** {@inheritDoc} */
- @Override protected Object processOutObject(int type) throws IgniteCheckedException {
+ @Override public PlatformTarget processOutObject(int type) throws IgniteCheckedException {
switch (type) {
case OP_WITH_ASYNC:
if (messaging.isAsync())
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
index 962a4c0..22a7fa2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx;
import org.apache.ignite.internal.binary.BinaryRawWriterEx;
import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformTarget;
import org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetService;
import org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetServiceImpl;
import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
@@ -144,7 +145,7 @@ public class PlatformServices extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader)
+ @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader)
throws IgniteCheckedException {
switch (type) {
case OP_DOTNET_DEPLOY: {
@@ -195,7 +196,7 @@ public class PlatformServices extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
+ @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
throws IgniteCheckedException {
switch (type) {
case OP_DOTNET_SERVICES: {
@@ -223,8 +224,8 @@ public class PlatformServices extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected Object processInObjectStreamOutObjectStream(int type, Object arg, BinaryRawReaderEx reader,
- BinaryRawWriterEx writer) throws IgniteCheckedException {
+ @Override public PlatformTarget processInObjectStreamOutObjectStream(int type, PlatformTarget arg,
+ BinaryRawReaderEx reader, BinaryRawWriterEx writer) throws IgniteCheckedException {
switch (type) {
case OP_INVOKE: {
assert arg != null;
@@ -260,7 +261,7 @@ public class PlatformServices extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+ @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
switch (type) {
case OP_DESCRIPTORS: {
Collection<ServiceDescriptor> descs = services.serviceDescriptors();
@@ -299,7 +300,7 @@ public class PlatformServices extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected Object processOutObject(int type) throws IgniteCheckedException {
+ @Override public PlatformTarget processOutObject(int type) throws IgniteCheckedException {
switch (type) {
case OP_WITH_ASYNC:
if (services.isAsync())
@@ -315,7 +316,7 @@ public class PlatformServices extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+ @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
switch (type) {
case OP_CANCEL_ALL:
services.cancelAll();
@@ -327,7 +328,7 @@ public class PlatformServices extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected Object processInStreamOutObject(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+ @Override public PlatformTarget processInStreamOutObject(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
switch (type) {
case OP_SERVICE_PROXY: {
String name = reader.readString();
@@ -343,14 +344,14 @@ public class PlatformServices extends PlatformAbstractTarget {
: new GridServiceProxy<>(services.clusterGroup(), name, Service.class, sticky,
platformCtx.kernalContext());
- return new ServiceProxyHolder(proxy, d.serviceClass());
+ return new ServiceProxyHolder(proxy, d.serviceClass(), platformContext());
}
}
return super.processInStreamOutObject(type, reader);
}
/** {@inheritDoc} */
- @Override protected IgniteInternalFuture currentFuture() throws IgniteCheckedException {
+ @Override public IgniteInternalFuture currentFuture() throws IgniteCheckedException {
return ((IgniteFutureImpl)servicesAsync.future()).internalFuture();
}
@@ -392,7 +393,7 @@ public class PlatformServices extends PlatformAbstractTarget {
* Proxy holder.
*/
@SuppressWarnings("unchecked")
- private static class ServiceProxyHolder {
+ private static class ServiceProxyHolder extends PlatformAbstractTarget {
/** */
private final Object proxy;
@@ -422,7 +423,9 @@ public class PlatformServices extends PlatformAbstractTarget {
* @param proxy Proxy object.
* @param clazz Proxy class.
*/
- private ServiceProxyHolder(Object proxy, Class clazz) {
+ private ServiceProxyHolder(Object proxy, Class clazz, PlatformContext ctx) {
+ super(ctx);
+
assert proxy != null;
assert clazz != null;
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
index 1b41712..3cee2b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
@@ -153,7 +153,7 @@ public class PlatformTransactions extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+ @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
switch (type) {
case OP_COMMIT:
tx(val).commit();
@@ -184,7 +184,7 @@ public class PlatformTransactions extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+ @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
long txId = reader.readLong();
final Transaction asyncTx = (Transaction)tx(txId).withAsync();
@@ -220,7 +220,7 @@ public class PlatformTransactions extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer) throws IgniteCheckedException {
+ @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer) throws IgniteCheckedException {
switch (type) {
case OP_START: {
TransactionConcurrency txConcurrency = TransactionConcurrency.fromOrdinal(reader.readInt());
@@ -245,7 +245,7 @@ public class PlatformTransactions extends PlatformAbstractTarget {
}
/** {@inheritDoc} */
- @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+ @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
switch (type) {
case OP_CACHE_CONFIG_PARAMETERS:
TransactionConfiguration txCfg = platformCtx.kernalContext().config().getTransactionConfiguration();
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java
index 5985d22..e81f4c6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java
@@ -20,8 +20,8 @@ package org.apache.ignite.internal.processors.platform.utils;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.binary.BinaryRawWriterEx;
-import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformTarget;
import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway;
import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
@@ -71,7 +71,7 @@ public class PlatformFutureUtils {
* @return Resulting listenable.
*/
public static PlatformListenable listen(final PlatformContext ctx, IgniteInternalFuture fut, final long futPtr,
- final int typ, PlatformAbstractTarget target) {
+ final int typ, PlatformTarget target) {
PlatformListenable listenable = getListenable(fut);
listen(ctx, listenable, futPtr, typ, null, target);
@@ -88,7 +88,7 @@ public class PlatformFutureUtils {
* @return Resulting listenable.
*/
public static PlatformListenable listen(final PlatformContext ctx, IgniteFuture fut, final long futPtr,
- final int typ, PlatformAbstractTarget target) {
+ final int typ, PlatformTarget target) {
PlatformListenable listenable = getListenable(fut);
listen(ctx, listenable, futPtr, typ, null, target);
@@ -107,7 +107,7 @@ public class PlatformFutureUtils {
* @return Resulting listenable.
*/
public static PlatformListenable listen(final PlatformContext ctx, IgniteInternalFuture fut, final long futPtr,
- final int typ, Writer writer, PlatformAbstractTarget target) {
+ final int typ, Writer writer, PlatformTarget target) {
PlatformListenable listenable = getListenable(fut);
listen(ctx, listenable, futPtr, typ, writer, target);
@@ -126,7 +126,7 @@ public class PlatformFutureUtils {
* @return Resulting listenable.
*/
public static PlatformListenable listen(final PlatformContext ctx, IgniteFuture fut, final long futPtr,
- final int typ, Writer writer, PlatformAbstractTarget target) {
+ final int typ, Writer writer, PlatformTarget target) {
PlatformListenable listenable = getListenable(fut);
listen(ctx, listenable, futPtr, typ, writer, target);
@@ -144,7 +144,7 @@ public class PlatformFutureUtils {
* @return Resulting listenable.
*/
public static PlatformListenable listen(final PlatformContext ctx, IgniteInternalFuture fut, final long futPtr,
- Writer writer, PlatformAbstractTarget target) {
+ Writer writer, PlatformTarget target) {
PlatformListenable listenable = getListenable(fut);
listen(ctx, listenable, futPtr, TYP_OBJ, writer, target);
@@ -183,7 +183,7 @@ public class PlatformFutureUtils {
*/
@SuppressWarnings("unchecked")
public static void listen(final PlatformContext ctx, PlatformListenable listenable, final long futPtr, final
- int typ, @Nullable final Writer writer, final PlatformAbstractTarget target) {
+ int typ, @Nullable final Writer writer, final PlatformTarget target) {
final PlatformCallbackGateway gate = ctx.gateway();
listenable.listen(new IgniteBiInClosure<Object, Throwable>() {
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformListenableTarget.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformListenableTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformListenableTarget.java
new file mode 100644
index 0000000..7d65913
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformListenableTarget.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform.utils;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
+import org.apache.ignite.internal.processors.platform.PlatformContext;
+
+/**
+ * Wraps listenable in a platform target.
+ */
+public class PlatformListenableTarget extends PlatformAbstractTarget {
+ /** */
+ private static final int OP_CANCEL = 1;
+
+ /** */
+ private static final int OP_IS_CANCELLED = 2;
+
+ /** Wrapped listenable */
+ private final PlatformListenable listenable;
+
+ /**
+ * Constructor.
+ *
+ * @param platformCtx Context.
+ */
+ public PlatformListenableTarget(PlatformListenable listenable, PlatformContext platformCtx) {
+ super(platformCtx);
+
+ assert listenable != null;
+
+ this.listenable = listenable;
+ }
+
+ /** {@inheritDoc} */
+ @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+ switch (type) {
+ case OP_CANCEL:
+ return listenable.cancel() ? TRUE : FALSE;
+
+ case OP_IS_CANCELLED:
+ return listenable.isCancelled() ? TRUE : FALSE;
+ }
+
+ return super.processInLongOutLong(type, val);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/cpp/jni/include/ignite/jni/exports.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/include/ignite/jni/exports.h b/modules/platforms/cpp/jni/include/ignite/jni/exports.h
index 3052435..a2e5cbb 100644
--- a/modules/platforms/cpp/jni/include/ignite/jni/exports.h
+++ b/modules/platforms/cpp/jni/include/ignite/jni/exports.h
@@ -80,9 +80,6 @@ extern "C" {
void IGNITE_CALL IgniteDestroyJvm(gcj::JniContext* ctx);
- bool IGNITE_CALL IgniteListenableCancel(gcj::JniContext* ctx, void* obj);
- bool IGNITE_CALL IgniteListenableIsCancelled(gcj::JniContext* ctx, void* obj);
-
void IGNITE_CALL IgniteSetConsoleHandler(gcj::ConsoleWriteHandler consoleHandler);
void IGNITE_CALL IgniteRemoveConsoleHandler(gcj::ConsoleWriteHandler consoleHandler);
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/cpp/jni/include/ignite/jni/java.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/include/ignite/jni/java.h b/modules/platforms/cpp/jni/include/ignite/jni/java.h
index 07df001..97e4412 100644
--- a/modules/platforms/cpp/jni/include/ignite/jni/java.h
+++ b/modules/platforms/cpp/jni/include/ignite/jni/java.h
@@ -286,10 +286,6 @@ namespace ignite
jmethodID m_PlatformUtils_reallocate;
jmethodID m_PlatformUtils_errData;
- jclass c_PlatformListenable;
- jmethodID m_PlatformListenable_cancel;
- jmethodID m_PlatformListenable_isCancelled;
-
/**
* Constructor.
*/
@@ -465,9 +461,6 @@ namespace ignite
jobject CacheOutOpQueryCursor(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL);
jobject CacheOutOpContinuousQuery(jobject obj, int type, long long memPtr);
- bool ListenableCancel(jobject obj);
- bool ListenableIsCancelled(jobject obj);
-
jobject Acquire(jobject obj);
void DestroyJvm();
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/cpp/jni/project/vs/module.def
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/project/vs/module.def b/modules/platforms/cpp/jni/project/vs/module.def
index e58ac3b..fb56dca 100644
--- a/modules/platforms/cpp/jni/project/vs/module.def
+++ b/modules/platforms/cpp/jni/project/vs/module.def
@@ -36,8 +36,6 @@ IgniteDestroyJvm @86
IgniteTargetOutObject @91
IgniteProcessorExtensions @97
IgniteProcessorAtomicLong @98
-IgniteListenableCancel @110
-IgniteListenableIsCancelled @111
IgniteProcessorCreateCacheFromConfig @114
IgniteProcessorGetOrCreateCacheFromConfig @115
IgniteProcessorGetIgniteConfiguration @116
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/cpp/jni/src/exports.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/src/exports.cpp b/modules/platforms/cpp/jni/src/exports.cpp
index dde98fb..b842c03 100644
--- a/modules/platforms/cpp/jni/src/exports.cpp
+++ b/modules/platforms/cpp/jni/src/exports.cpp
@@ -214,14 +214,6 @@ extern "C" {
ctx->DestroyJvm();
}
- bool IGNITE_CALL IgniteListenableCancel(gcj::JniContext* ctx, void* obj) {
- return ctx->ListenableCancel(static_cast<jobject>(obj));
- }
-
- bool IGNITE_CALL IgniteListenableIsCancelled(gcj::JniContext* ctx, void* obj) {
- return ctx->ListenableIsCancelled(static_cast<jobject>(obj));
- }
-
void IGNITE_CALL IgniteSetConsoleHandler(gcj::ConsoleWriteHandler consoleHandler) {
gcj::JniContext::SetConsoleHandler(consoleHandler);
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/cpp/jni/src/java.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/src/java.cpp b/modules/platforms/cpp/jni/src/java.cpp
index 2d3cf72..9626fbb 100644
--- a/modules/platforms/cpp/jni/src/java.cpp
+++ b/modules/platforms/cpp/jni/src/java.cpp
@@ -194,33 +194,33 @@ namespace ignite
const char* C_PLATFORM_PROCESSOR = "org/apache/ignite/internal/processors/platform/PlatformProcessor";
JniMethod M_PLATFORM_PROCESSOR_RELEASE_START = JniMethod("releaseStart", "()V", false);
- JniMethod M_PLATFORM_PROCESSOR_PROJECTION = JniMethod("projection", "()Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
- JniMethod M_PLATFORM_PROCESSOR_CACHE = JniMethod("cache", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
- JniMethod M_PLATFORM_PROCESSOR_CREATE_CACHE = JniMethod("createCache", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
- JniMethod M_PLATFORM_PROCESSOR_GET_OR_CREATE_CACHE = JniMethod("getOrCreateCache", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
- JniMethod M_PLATFORM_PROCESSOR_CREATE_CACHE_FROM_CONFIG = JniMethod("createCacheFromConfig", "(J)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
- JniMethod M_PLATFORM_PROCESSOR_GET_OR_CREATE_CACHE_FROM_CONFIG = JniMethod("getOrCreateCacheFromConfig", "(J)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
- JniMethod M_PLATFORM_PROCESSOR_CREATE_NEAR_CACHE = JniMethod("createNearCache", "(Ljava/lang/String;J)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
- JniMethod M_PLATFORM_PROCESSOR_GET_OR_CREATE_NEAR_CACHE = JniMethod("getOrCreateNearCache", "(Ljava/lang/String;J)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+ JniMethod M_PLATFORM_PROCESSOR_PROJECTION = JniMethod("projection", "()Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+ JniMethod M_PLATFORM_PROCESSOR_CACHE = JniMethod("cache", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+ JniMethod M_PLATFORM_PROCESSOR_CREATE_CACHE = JniMethod("createCache", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+ JniMethod M_PLATFORM_PROCESSOR_GET_OR_CREATE_CACHE = JniMethod("getOrCreateCache", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+ JniMethod M_PLATFORM_PROCESSOR_CREATE_CACHE_FROM_CONFIG = JniMethod("createCacheFromConfig", "(J)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+ JniMethod M_PLATFORM_PROCESSOR_GET_OR_CREATE_CACHE_FROM_CONFIG = JniMethod("getOrCreateCacheFromConfig", "(J)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+ JniMethod M_PLATFORM_PROCESSOR_CREATE_NEAR_CACHE = JniMethod("createNearCache", "(Ljava/lang/String;J)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+ JniMethod M_PLATFORM_PROCESSOR_GET_OR_CREATE_NEAR_CACHE = JniMethod("getOrCreateNearCache", "(Ljava/lang/String;J)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
JniMethod M_PLATFORM_PROCESSOR_DESTROY_CACHE = JniMethod("destroyCache", "(Ljava/lang/String;)V", false);
- JniMethod M_PLATFORM_PROCESSOR_AFFINITY = JniMethod("affinity", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
- JniMethod M_PLATFORM_PROCESSOR_DATA_STREAMER = JniMethod("dataStreamer", "(Ljava/lang/String;Z)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
- JniMethod M_PLATFORM_PROCESSOR_TRANSACTIONS = JniMethod("transactions", "()Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
- JniMethod M_PLATFORM_PROCESSOR_COMPUTE = JniMethod("compute", "(Lorg/apache/ignite/internal/processors/platform/PlatformTarget;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
- JniMethod M_PLATFORM_PROCESSOR_MESSAGE = JniMethod("message", "(Lorg/apache/ignite/internal/processors/platform/PlatformTarget;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
- JniMethod M_PLATFORM_PROCESSOR_EVENTS = JniMethod("events", "(Lorg/apache/ignite/internal/processors/platform/PlatformTarget;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
- JniMethod M_PLATFORM_PROCESSOR_SERVICES = JniMethod("services", "(Lorg/apache/ignite/internal/processors/platform/PlatformTarget;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
- JniMethod M_PLATFORM_PROCESSOR_EXTENSIONS = JniMethod("extensions", "()Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
- JniMethod M_PLATFORM_PROCESSOR_ATOMIC_LONG = JniMethod("atomicLong", "(Ljava/lang/String;JZ)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
- JniMethod M_PLATFORM_PROCESSOR_ATOMIC_SEQUENCE = JniMethod("atomicSequence", "(Ljava/lang/String;JZ)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
- JniMethod M_PLATFORM_PROCESSOR_ATOMIC_REFERENCE = JniMethod("atomicReference", "(Ljava/lang/String;JZ)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+ JniMethod M_PLATFORM_PROCESSOR_AFFINITY = JniMethod("affinity", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+ JniMethod M_PLATFORM_PROCESSOR_DATA_STREAMER = JniMethod("dataStreamer", "(Ljava/lang/String;Z)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+ JniMethod M_PLATFORM_PROCESSOR_TRANSACTIONS = JniMethod("transactions", "()Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+ JniMethod M_PLATFORM_PROCESSOR_COMPUTE = JniMethod("compute", "(Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+ JniMethod M_PLATFORM_PROCESSOR_MESSAGE = JniMethod("message", "(Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+ JniMethod M_PLATFORM_PROCESSOR_EVENTS = JniMethod("events", "(Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+ JniMethod M_PLATFORM_PROCESSOR_SERVICES = JniMethod("services", "(Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+ JniMethod M_PLATFORM_PROCESSOR_EXTENSIONS = JniMethod("extensions", "()Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+ JniMethod M_PLATFORM_PROCESSOR_ATOMIC_LONG = JniMethod("atomicLong", "(Ljava/lang/String;JZ)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+ JniMethod M_PLATFORM_PROCESSOR_ATOMIC_SEQUENCE = JniMethod("atomicSequence", "(Ljava/lang/String;JZ)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+ JniMethod M_PLATFORM_PROCESSOR_ATOMIC_REFERENCE = JniMethod("atomicReference", "(Ljava/lang/String;JZ)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
JniMethod M_PLATFORM_PROCESSOR_GET_IGNITE_CONFIGURATION = JniMethod("getIgniteConfiguration", "(J)V", false);
JniMethod M_PLATFORM_PROCESSOR_GET_CACHE_NAMES = JniMethod("getCacheNames", "(J)V", false);
JniMethod M_PLATFORM_PROCESSOR_LOGGER_IS_LEVEL_ENABLED = JniMethod("loggerIsLevelEnabled", "(I)Z", false);
JniMethod M_PLATFORM_PROCESSOR_LOGGER_LOG = JniMethod("loggerLog", "(ILjava/lang/String;Ljava/lang/String;Ljava/lang/String;)V", false);
- JniMethod M_PLATFORM_PROCESSOR_BINARY_PROCESSOR = JniMethod("binaryProcessor", "()Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+ JniMethod M_PLATFORM_PROCESSOR_BINARY_PROCESSOR = JniMethod("binaryProcessor", "()Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
- const char* C_PLATFORM_TARGET = "org/apache/ignite/internal/processors/platform/PlatformTarget";
+ const char* C_PLATFORM_TARGET = "org/apache/ignite/internal/processors/platform/PlatformTargetProxy";
JniMethod M_PLATFORM_TARGET_IN_LONG_OUT_LONG = JniMethod("inLongOutLong", "(IJ)J", false);
JniMethod M_PLATFORM_TARGET_IN_STREAM_OUT_LONG = JniMethod("inStreamOutLong", "(IJ)J", false);
JniMethod M_PLATFORM_TARGET_IN_STREAM_OUT_OBJECT = JniMethod("inStreamOutObject", "(IJ)Ljava/lang/Object;", false);
@@ -260,7 +260,7 @@ namespace ignite
JniMethod M_PLATFORM_CALLBACK_UTILS_CONTINUOUS_QUERY_FILTER_RELEASE = JniMethod("continuousQueryFilterRelease", "(JJ)V", true);
JniMethod M_PLATFORM_CALLBACK_UTILS_DATA_STREAMER_TOPOLOGY_UPDATE = JniMethod("dataStreamerTopologyUpdate", "(JJJI)V", true);
- JniMethod M_PLATFORM_CALLBACK_UTILS_DATA_STREAMER_STREAM_RECEIVER_INVOKE = JniMethod("dataStreamerStreamReceiverInvoke", "(JJLjava/lang/Object;JZ)V", true);
+ JniMethod M_PLATFORM_CALLBACK_UTILS_DATA_STREAMER_STREAM_RECEIVER_INVOKE = JniMethod("dataStreamerStreamReceiverInvoke", "(JJLorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;JZ)V", true);
JniMethod M_PLATFORM_CALLBACK_UTILS_FUTURE_BYTE_RES = JniMethod("futureByteResult", "(JJI)V", true);
JniMethod M_PLATFORM_CALLBACK_UTILS_FUTURE_BOOL_RES = JniMethod("futureBoolResult", "(JJI)V", true);
@@ -307,7 +307,7 @@ namespace ignite
JniMethod M_PLATFORM_CALLBACK_UTILS_LOGGER_LOG = JniMethod("loggerLog", "(JILjava/lang/String;Ljava/lang/String;Ljava/lang/String;J)V", true);
JniMethod M_PLATFORM_CALLBACK_UTILS_LOGGER_IS_LEVEL_ENABLED = JniMethod("loggerIsLevelEnabled", "(JI)Z", true);
- JniMethod M_PLATFORM_CALLBACK_UTILS_AFFINITY_FUNCTION_INIT = JniMethod("affinityFunctionInit", "(JJLorg/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunctionTarget;)J", true);
+ JniMethod M_PLATFORM_CALLBACK_UTILS_AFFINITY_FUNCTION_INIT = JniMethod("affinityFunctionInit", "(JJLorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;)J", true);
JniMethod M_PLATFORM_CALLBACK_UTILS_AFFINITY_FUNCTION_PARTITION = JniMethod("affinityFunctionPartition", "(JJJ)I", true);
JniMethod M_PLATFORM_CALLBACK_UTILS_AFFINITY_FUNCTION_ASSIGN_PARTITIONS = JniMethod("affinityFunctionAssignPartitions", "(JJJJ)V", true);
JniMethod M_PLATFORM_CALLBACK_UTILS_AFFINITY_FUNCTION_REMOVE_NODE = JniMethod("affinityFunctionRemoveNode", "(JJJ)V", true);
@@ -327,10 +327,6 @@ namespace ignite
JniMethod M_PLATFORM_IGNITION_STOP = JniMethod("stop", "(Ljava/lang/String;Z)Z", true);
JniMethod M_PLATFORM_IGNITION_STOP_ALL = JniMethod("stopAll", "(Z)V", true);
- const char* C_PLATFORM_LISTENABLE = "org/apache/ignite/internal/processors/platform/utils/PlatformListenable";
- JniMethod M_PLATFORM_LISTENABLE_CANCEL = JniMethod("cancel", "()Z", false);
- JniMethod M_PLATFORM_LISTENABLE_IS_CANCELED = JniMethod("isCancelled", "()Z", false);
-
/* STATIC STATE. */
gcc::CriticalSection JVM_LOCK;
gcc::CriticalSection CONSOLE_LOCK;
@@ -552,10 +548,6 @@ namespace ignite
m_PlatformUtils_reallocate = FindMethod(env, c_PlatformUtils, M_PLATFORM_UTILS_REALLOC);
m_PlatformUtils_errData = FindMethod(env, c_PlatformUtils, M_PLATFORM_UTILS_ERR_DATA);
- c_PlatformListenable = FindClass(env, C_PLATFORM_LISTENABLE);
- m_PlatformListenable_cancel = FindMethod(env, c_PlatformListenable, M_PLATFORM_LISTENABLE_CANCEL);
- m_PlatformListenable_isCancelled = FindMethod(env, c_PlatformListenable, M_PLATFORM_LISTENABLE_IS_CANCELED);
-
// Find utility classes which are not used from context, but are still required in other places.
CheckClass(env, C_PLATFORM_NO_CALLBACK_EXCEPTION);
}
@@ -1447,28 +1439,6 @@ namespace ignite
return LocalToGlobal(env, res);
}
- bool JniContext::ListenableCancel(jobject obj)
- {
- JNIEnv* env = Attach();
-
- jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformListenable_cancel);
-
- ExceptionCheck(env);
-
- return res != 0;;
- }
-
- bool JniContext::ListenableIsCancelled(jobject obj)
- {
- JNIEnv* env = Attach();
-
- jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformListenable_isCancelled);
-
- ExceptionCheck(env);
-
- return res != 0;;
- }
-
jobject JniContext::Acquire(jobject obj)
{
if (obj) {
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index f945efe..6421b8c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -181,6 +181,7 @@
<Compile Include="Impl\Cache\Expiry\ExpiryPolicyFactory.cs" />
<Compile Include="Impl\Cache\Expiry\ExpiryPolicySerializer.cs" />
<Compile Include="Impl\Cache\ICacheLockInternal.cs" />
+ <Compile Include="Impl\Common\Listenable.cs" />
<Compile Include="Impl\Common\Platform.cs" />
<Compile Include="Impl\Binary\UserSerializerProxy.cs" />
<Compile Include="Impl\Cache\Affinity\AffinityFunctionSerializer.cs" />
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
index 68bd9d4..50102a7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
@@ -24,7 +24,6 @@ namespace Apache.Ignite.Core.Impl.Common
using System.Threading.Tasks;
using Apache.Ignite.Core.Common;
using Apache.Ignite.Core.Impl.Binary.IO;
- using Apache.Ignite.Core.Impl.Unmanaged;
/// <summary>
/// Grid future implementation.
@@ -40,7 +39,7 @@ namespace Apache.Ignite.Core.Impl.Common
private readonly TaskCompletionSource<T> _taskCompletionSource = new TaskCompletionSource<T>();
/** */
- private volatile IUnmanagedTarget _unmanagedTarget;
+ private volatile Listenable _listenable;
/// <summary>
/// Constructor.
@@ -84,7 +83,7 @@ namespace Apache.Ignite.Core.Impl.Common
/// <param name="cancellationToken">The cancellation token.</param>
public Task<T> GetTask(CancellationToken cancellationToken)
{
- Debug.Assert(_unmanagedTarget != null);
+ Debug.Assert(_listenable != null);
// OnTokenCancel will fire even if cancellationToken is already cancelled.
cancellationToken.Register(OnTokenCancel);
@@ -169,11 +168,11 @@ namespace Apache.Ignite.Core.Impl.Common
/// <summary>
/// Sets unmanaged future target for cancellation.
/// </summary>
- internal void SetTarget(IUnmanagedTarget target)
+ internal void SetTarget(Listenable target)
{
Debug.Assert(target != null);
- _unmanagedTarget = target;
+ _listenable = target;
}
/// <summary>
@@ -181,8 +180,8 @@ namespace Apache.Ignite.Core.Impl.Common
/// </summary>
private void OnTokenCancel()
{
- if (_unmanagedTarget != null)
- UnmanagedUtils.ListenableCancel(_unmanagedTarget);
+ if (_listenable != null)
+ _listenable.Cancel();
}
}
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Listenable.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Listenable.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Listenable.cs
new file mode 100644
index 0000000..6da98ab
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Listenable.cs
@@ -0,0 +1,49 @@
+\ufeff/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Common
+{
+ using Apache.Ignite.Core.Impl.Binary;
+ using Apache.Ignite.Core.Impl.Unmanaged;
+
+ /// <summary>
+ /// Platform listenable.
+ /// </summary>
+ internal class Listenable : PlatformTarget
+ {
+ /** */
+ private const int OpCancel = 1;
+
+ /// <summary>
+ /// Initializes a new instance of the <see cref="Listenable"/> class.
+ /// </summary>
+ /// <param name="target">Target.</param>
+ /// <param name="marsh">Marshaller.</param>
+ public Listenable(IUnmanagedTarget target, Marshaller marsh) : base(target, marsh)
+ {
+ // No-op.
+ }
+
+ /// <summary>
+ /// Cancels the listenable.
+ /// </summary>
+ public void Cancel()
+ {
+ DoOutInOp(OpCancel);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs
index bc7c7d9..d36caf3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs
@@ -195,7 +195,7 @@ namespace Apache.Ignite.Core.Impl.Compute
var future = holder.Future;
- future.SetTarget(futTarget);
+ future.SetTarget(new Listenable(futTarget, Marshaller));
return future;
}
@@ -550,7 +550,7 @@ namespace Apache.Ignite.Core.Impl.Compute
writeAction(writer);
});
- holder.Future.SetTarget(futTarget);
+ holder.Future.SetTarget(new Listenable(futTarget, Marshaller));
}
catch (Exception e)
{
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
index f4a07f6..9cf2a6c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
@@ -869,7 +869,7 @@ namespace Apache.Ignite.Core.Impl
throw;
}
- fut.SetTarget(futTarget);
+ fut.SetTarget(new Listenable(futTarget, _marsh));
return fut;
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
index c746866..c4f3e19 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
@@ -167,14 +167,6 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
[DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDestroyJvm")]
public static extern void DestroyJvm(void* ctx);
- [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteListenableCancel")]
- [return: MarshalAs(UnmanagedType.U1)]
- public static extern bool ListenableCancel(void* ctx, void* target);
-
- [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteListenableIsCancelled")]
- [return: MarshalAs(UnmanagedType.U1)]
- public static extern bool ListenableIsCancelled(void* ctx, void* target);
-
[DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteSetConsoleHandler")]
public static extern void SetConsoleHandler(void* consoleHandler);
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
index f36c35f..0a2a1f0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
@@ -516,11 +516,6 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
JNI.DestroyJvm(ctx);
}
- internal static bool ListenableCancel(IUnmanagedTarget target)
- {
- return JNI.ListenableCancel(target.Context, target.Target);
- }
-
#endregion
}
}