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 2015/05/18 11:31:10 UTC

[01/29] incubator-ignite git commit: # ignite-709_3 do not create system caches on client nodes

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-709_2 ed76bf0c5 -> c1fd8e706


# ignite-709_3 do not create system caches on client nodes


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

Branch: refs/heads/ignite-709_2
Commit: 63261f5866695ddfbbca3f9d5508f4d7964263dc
Parents: 4ba5671
Author: sboikov <sb...@gridgain.com>
Authored: Thu May 14 13:07:44 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu May 14 13:35:34 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/IgniteEx.java    |  3 +-
 .../apache/ignite/internal/IgniteKernal.java    |  3 +-
 .../org/apache/ignite/internal/IgnitionEx.java  | 16 ++--
 .../processors/cache/GridCacheProcessor.java    | 94 +++++++++++++++-----
 .../processors/cache/GridCacheUtils.java        | 25 ++++--
 .../GridDhtPartitionsExchangeFuture.java        | 19 +++-
 6 files changed, 112 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/63261f58/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
index 4845d51..bc7e722 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
@@ -35,8 +35,9 @@ public interface IgniteEx extends Ignite {
      * Gets utility cache.
      *
      * @return Utility cache.
+     * @throws IgniteCheckedException If failed.
      */
-    public <K extends GridCacheUtilityKey, V> IgniteInternalCache<K, V> utilityCache();
+    public <K extends GridCacheUtilityKey, V> IgniteInternalCache<K, V> utilityCache() throws IgniteCheckedException;
 
     /**
      * Gets the cache instance for the given name if one is configured or

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/63261f58/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 d98b023..1c68a82 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
@@ -2462,7 +2462,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Override public <K extends GridCacheUtilityKey, V> IgniteInternalCache<K, V> utilityCache() {
+    @Override public <K extends GridCacheUtilityKey, V> IgniteInternalCache<K, V> utilityCache()
+        throws IgniteCheckedException {
         guard();
 
         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/63261f58/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 0807d09..9c7340e 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
@@ -1822,20 +1822,14 @@ public class IgnitionEx {
         public void initializeDefaultCacheConfiguration(IgniteConfiguration cfg) throws IgniteCheckedException {
             List<CacheConfiguration> cacheCfgs = new ArrayList<>();
 
-            boolean clientDisco = cfg.getDiscoverySpi() instanceof TcpClientDiscoverySpi;
+            cacheCfgs.add(marshallerSystemCache());
 
-            // Add marshaller and utility caches.
-            if (!clientDisco) {
-                cacheCfgs.add(marshallerSystemCache());
-
-                cacheCfgs.add(utilitySystemCache());
-            }
+            cacheCfgs.add(utilitySystemCache());
 
             if (IgniteComponentType.HADOOP.inClassPath())
                 cacheCfgs.add(CU.hadoopSystemCache());
 
-            if (cfg.getAtomicConfiguration() != null && !clientDisco)
-                cacheCfgs.add(atomicsSystemCache(cfg.getAtomicConfiguration()));
+            cacheCfgs.add(atomicsSystemCache(cfg.getAtomicConfiguration()));
 
             CacheConfiguration[] userCaches = cfg.getCacheConfiguration();
 
@@ -2021,7 +2015,7 @@ public class IgnitionEx {
             cache.setRebalanceMode(SYNC);
             cache.setWriteSynchronizationMode(FULL_SYNC);
             cache.setAffinity(new RendezvousAffinityFunction(false, 20));
-            cache.setNodeFilter(CacheConfiguration.ALL_NODES);
+            cache.setNodeFilter(CacheConfiguration.SERVER_NODES);
             cache.setStartSize(300);
 
             return cache;
@@ -2042,7 +2036,7 @@ public class IgnitionEx {
             cache.setRebalanceMode(SYNC);
             cache.setWriteSynchronizationMode(FULL_SYNC);
             cache.setAffinity(new RendezvousAffinityFunction(false, 100));
-            cache.setNodeFilter(CacheConfiguration.ALL_NODES);
+            cache.setNodeFilter(CacheConfiguration.SERVER_NODES);
 
             return cache;
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/63261f58/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 0e1a9c2..86d80c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -733,13 +733,26 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             cacheStartedLatch.countDown();
         }
 
-        ctx.marshallerContext().onMarshallerCacheStarted(ctx);
+        if (marshallerCache() == null) {
+            assert ctx.config().isClientMode() : "Marshaller cache is missed on server node.";
 
-        marshallerCache().context().preloader().syncFuture().listen(new CIX1<IgniteInternalFuture<?>>() {
-            @Override public void applyx(IgniteInternalFuture<?> f) throws IgniteCheckedException {
-                ctx.marshallerContext().onMarshallerCachePreloaded(ctx);
-            }
-        });
+            IgniteInternalFuture<?> fut = startCacheAsync(CU.MARSH_CACHE_NAME, true);
+
+            assert fut != null;
+
+            fut.listen(new CI1<IgniteInternalFuture<?>>() {
+                @Override public void apply(IgniteInternalFuture<?> fut) {
+                    try {
+                        marshallerCacheCallbacks();
+                    }
+                    catch (IgniteCheckedException e) {
+                        U.error(log, "Failed to initialize marshaller context.", e);
+                    }
+                }
+            });
+        }
+        else
+            marshallerCacheCallbacks();
 
         // Must call onKernalStart on shared managers after creation of fetched caches.
         for (GridCacheSharedManager<?, ?> mgr : sharedCtx.managers())
@@ -788,6 +801,20 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         ctx.cacheObjects().onCacheProcessorStarted();
     }
 
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    private void marshallerCacheCallbacks() throws IgniteCheckedException {
+        ctx.marshallerContext().onMarshallerCacheStarted(ctx);
+
+        marshallerCache().context().preloader().syncFuture().listen(new CIX1<IgniteInternalFuture<?>>() {
+            @Override
+            public void applyx(IgniteInternalFuture<?> f) throws IgniteCheckedException {
+                ctx.marshallerContext().onMarshallerCachePreloaded(ctx);
+            }
+        });
+    }
+
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void stop(boolean cancel) throws IgniteCheckedException {
@@ -2499,18 +2526,19 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * Gets utility cache.
      *
      * @return Utility cache.
+     * @throws IgniteCheckedException If failed.
      */
-    public <K, V> GridCacheAdapter<K, V> utilityCache() {
-        return internalCache(CU.UTILITY_CACHE_NAME);
-    }
+    public <K, V> GridCacheAdapter<K, V> utilityCache() throws IgniteCheckedException {
+        GridCacheAdapter<K, V> cache = internalCache(CU.UTILITY_CACHE_NAME);
 
-    /**
-     * Gets utility cache for atomic data structures.
-     *
-     * @return Utility cache for atomic data structures.
-     */
-    public <K, V> IgniteInternalCache<K, V> atomicsCache() {
-        return cache(CU.ATOMICS_CACHE_NAME);
+        if (cache != null)
+            return cache;
+
+        assert ctx.config().isClientMode() : "Utility cache is missed on server node.";
+
+        getOrStartCache(CU.UTILITY_CACHE_NAME);
+
+        return internalCache(CU.UTILITY_CACHE_NAME);
     }
 
     /**
@@ -2590,6 +2618,31 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If failed.
      */
     private IgniteCache startJCache(String cacheName, boolean failIfNotStarted) throws IgniteCheckedException {
+        IgniteInternalFuture<?> fut = startCacheAsync(cacheName, failIfNotStarted);
+
+        if (fut != null) {
+            fut.get();
+
+            String masked = maskNull(cacheName);
+
+            IgniteCache cache = jCacheProxies.get(masked);
+
+            if (cache == null && failIfNotStarted)
+                throw new IllegalArgumentException("Cache is not started: " + cacheName);
+
+            return cache;
+        }
+
+        return null;
+    }
+
+    /**
+     * @param cacheName Cache name.
+     * @param failIfNotStarted If {@code true} throws {@link IllegalArgumentException} if cache is not started,
+     *        otherwise returns {@code null} in this case.
+     * @return Future.
+     */
+    @Nullable private IgniteInternalFuture<?> startCacheAsync(String cacheName, boolean failIfNotStarted) {
         String masked = maskNull(cacheName);
 
         DynamicCacheDescriptor desc = registeredCaches.get(masked);
@@ -2617,14 +2670,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         req.clientStartOnly(true);
 
-        F.first(initiateCacheChanges(F.asList(req))).get();
-
-        IgniteCache cache = jCacheProxies.get(masked);
-
-        if (cache == null && failIfNotStarted)
-            throw new IllegalArgumentException("Cache is not started: " + cacheName);
-
-        return cache;
+        return F.first(initiateCacheChanges(F.asList(req)));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/63261f58/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 549f42f..e502b54 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
@@ -651,7 +651,7 @@ public class GridCacheUtils {
      * @return Oldest node for the given topology version.
      */
     public static ClusterNode oldest(GridCacheSharedContext cctx, AffinityTopologyVersion topOrder) {
-        ClusterNode oldest = null;
+        ClusterNode oldest = oldest(aliveCacheNodes(cctx, topOrder));
 
         for (ClusterNode n : aliveCacheNodes(cctx, topOrder)) {
             if (oldest == null || n.order() < oldest.order())
@@ -665,6 +665,21 @@ public class GridCacheUtils {
     }
 
     /**
+     * @param nodes Nodes.
+     * @return Oldest node for the given topology version.
+     */
+    @Nullable public static ClusterNode oldest(Collection<ClusterNode> nodes) {
+        ClusterNode oldest = null;
+
+        for (ClusterNode n : nodes) {
+            if (oldest == null || n.order() < oldest.order())
+                oldest = n;
+        }
+
+        return oldest;
+    }
+
+    /**
      * @return Empty filter.
      */
     @SuppressWarnings({"unchecked"})
@@ -1451,13 +1466,7 @@ public class GridCacheUtils {
     }
 
     /**
-     * @return Cache ID for utility cache.
-     */
-    public static int utilityCacheId() {
-        return cacheId(UTILITY_CACHE_NAME);
-    }
-
-    /**
+     * @param cacheName Cache name.
      * @return Cache ID.
      */
     public static int cacheId(String cacheName) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/63261f58/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 45d332c..120c537 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
@@ -221,12 +221,22 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
         log = cctx.logger(getClass());
 
+        initFut = new GridFutureAdapter<>();
+
         // Grab all nodes with order of equal or less than last joined node.
-        oldestNode.set(CU.oldest(cctx, exchId.topologyVersion()));
+        Collection<ClusterNode> nodes = CU.aliveCacheNodes(cctx, exchId.topologyVersion());
 
-        assert oldestNode.get() != null;
+        if (nodes.isEmpty()) {
+            initFut.onDone(true);
 
-        initFut = new GridFutureAdapter<>();
+            onDone(exchId.topologyVersion());
+
+            return;
+        }
+
+        oldestNode.set(CU.oldest(nodes));
+
+        assert oldestNode.get() != null;
 
         if (log.isDebugEnabled())
             log.debug("Creating exchange future [localNode=" + cctx.localNodeId() +
@@ -444,6 +454,9 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
      * @throws IgniteInterruptedCheckedException If interrupted.
      */
     public void init() throws IgniteInterruptedCheckedException {
+        if (isDone())
+            return;
+
         assert oldestNode.get() != null;
 
         if (init.compareAndSet(false, true)) {


[09/29] incubator-ignite git commit: # ignite-709_3 do not create system caches on client nodes

Posted by sb...@apache.org.
# ignite-709_3 do not create system caches on client nodes


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

Branch: refs/heads/ignite-709_2
Commit: fb1d79cf17a8e4f27ebbfbf141ce6cfb3cda1ff6
Parents: db06cd3
Author: sboikov <sb...@gridgain.com>
Authored: Thu May 14 17:47:21 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu May 14 17:47:21 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    | 32 +++++++++++++-------
 .../preloader/GridDhtPartitionDemandPool.java   |  4 +++
 2 files changed, 25 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fb1d79cf/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 8771515..9319b45 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -734,12 +734,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         }
 
         if (marshallerCache() == null) {
-            assert ctx.config().isClientMode() : "Marshaller cache is missed on server node.";
-
-            // On client node use near-only marshaller cache.
-            IgniteInternalFuture<?> fut = startCacheAsync(CU.MARSH_CACHE_NAME, new NearCacheConfiguration(), false);
-
-            assert fut != null;
+            IgniteInternalFuture<?> fut = marshallerCacheAsync();
 
             fut.listen(new CI1<IgniteInternalFuture<?>>() {
                 @Override public void apply(IgniteInternalFuture<?> fut) {
@@ -790,12 +785,14 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         // Wait for caches in SYNC preload mode.
         for (GridCacheAdapter<?, ?> cache : caches.values()) {
-            CacheConfiguration cfg = cache.configuration();
+            if (cache.context().started()) {
+                CacheConfiguration cfg = cache.configuration();
 
-            if (cfg.getRebalanceMode() == SYNC) {
-                if (cfg.getCacheMode() == REPLICATED ||
-                    (cfg.getCacheMode() == PARTITIONED && cfg.getRebalanceDelay() >= 0))
-                    cache.preloader().syncFuture().get();
+                if (cfg.getRebalanceMode() == SYNC) {
+                    if (cfg.getCacheMode() == REPLICATED ||
+                        (cfg.getCacheMode() == PARTITIONED && cfg.getRebalanceDelay() >= 0))
+                        cache.preloader().syncFuture().get();
+                }
             }
         }
 
@@ -2553,6 +2550,19 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @return Utility cache start future.
+     */
+    public IgniteInternalFuture<?> marshallerCacheAsync() {
+        if (internalCache(CU.MARSH_CACHE_NAME) != null)
+            return new GridFinishedFuture<>();
+
+        assert ctx.config().isClientMode() : "Marshaller cache is missed on server node.";
+
+        // On client node use near-only marshaller cache.
+        return startCacheAsync(CU.MARSH_CACHE_NAME, new NearCacheConfiguration(), false);
+    }
+
+    /**
      * @param name Cache name.
      * @param <K> type of keys.
      * @param <V> type of values.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fb1d79cf/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
index 633f237..ba2af9e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
@@ -824,6 +824,10 @@ public class GridDhtPartitionDemandPool<K, V> {
                         log.debug("Waiting for marshaller cache preload [cacheName=" + cctx.name() + ']');
 
                     try {
+                        cctx.kernalContext().cache().marshallerCacheAsync().get();
+
+                        cctx.kernalContext().cache().marshallerCache().context().awaitStarted();
+
                         cctx.kernalContext().cache().marshallerCache().preloader().syncFuture().get();
                     }
                     catch (IgniteInterruptedCheckedException ignored) {


[17/29] incubator-ignite git commit: # ignite-709_3

Posted by sb...@apache.org.
# ignite-709_3


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

Branch: refs/heads/ignite-709_2
Commit: 33459b1960c5d551a90fa2b57e1b1ab9e38293f9
Parents: ef6dd41
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 15 17:36:21 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 15 17:36:21 2015 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         |  4 +-
 .../processors/cache/GridCacheProcessor.java    |  2 +-
 .../IgniteDynamicClientCacheStartSelfTest.java  | 70 ++++++++++++++++++++
 .../igfs/IgfsClientCacheSelfTest.java           |  3 +-
 4 files changed, 75 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/33459b19/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 eac96b2..f81d471 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
@@ -2538,8 +2538,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             if (node.isDaemon())
                 return false;
 
-            if (nearEnabled && CU.affinityNode(node, cacheFilter))
-                return true;
+            if (CU.affinityNode(node, cacheFilter))
+                return nearEnabled;
 
             Boolean near = clientNodes.get(node.id());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/33459b19/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 9469e65..6295d28 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -622,7 +622,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 ctx.discovery().setCacheFilter(
                     cfg.getName(),
                     cfg.getNodeFilter(),
-                    cfg.getNearConfiguration() != null,
+                    cfg.getNearConfiguration() != null && cfg.getCacheMode() == PARTITIONED,
                     cfg.getCacheMode() == LOCAL);
 
                 ctx.discovery().addClientNode(cfg.getName(),

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/33459b19/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
index 9745ad8..24935c7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
@@ -32,6 +32,8 @@ import org.apache.ignite.testframework.junits.common.*;
 
 import java.util.concurrent.*;
 
+import static org.apache.ignite.cache.CacheMode.*;
+
 /**
  * Tests that cache specified in configuration start on client nodes.
  */
@@ -153,6 +155,74 @@ public class IgniteDynamicClientCacheStartSelfTest extends GridCommonAbstractTes
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testReplicatedCacheClient() throws Exception {
+        ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(REPLICATED);
+
+        final String cacheName = null;
+
+        Ignite ignite0 = startGrid(0);
+
+        checkCache(ignite0, cacheName, true, false);
+
+        client = true;
+
+        final Ignite ignite1 = startGrid(1);
+
+        checkCache(ignite1, cacheName, false, false);
+
+        ccfg.setNearConfiguration(new NearCacheConfiguration());
+
+        Ignite ignite2 = startGrid(2);
+
+        checkCache(ignite2, cacheName, false, true);
+
+        ccfg = null;
+
+        Ignite ignite3 = startGrid(3);
+
+        checkNoCache(ignite3, cacheName);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReplicatedWithNearCacheClient() throws Exception {
+        ccfg = new CacheConfiguration();
+
+        ccfg.setNearConfiguration(new NearCacheConfiguration());
+
+        ccfg.setCacheMode(REPLICATED);
+
+        final String cacheName = null;
+
+        Ignite ignite0 = startGrid(0);
+
+        checkCache(ignite0, cacheName, true, false);
+
+        client = true;
+
+        final Ignite ignite1 = startGrid(1);
+
+        checkCache(ignite1, cacheName, false, true);
+
+        ccfg.setNearConfiguration(null);
+
+        Ignite ignite2 = startGrid(2);
+
+        checkCache(ignite2, cacheName, false, false);
+
+        ccfg = null;
+
+        Ignite ignite3 = startGrid(3);
+
+        checkNoCache(ignite3, cacheName);
+    }
+
+    /**
      * @param ignite Node.
      * @param cacheName Cache name
      * @param srv {@code True} if server cache is expected.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/33459b19/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
index d983302..02166c4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
@@ -114,11 +114,12 @@ public class IgfsClientCacheSelfTest extends IgfsAbstractSelfTest {
 
         cacheCfg.setName(cacheName);
 
+        cacheCfg.setNearConfiguration(null);
+
         if (META_CACHE_NAME.equals(cacheName))
             cacheCfg.setCacheMode(REPLICATED);
         else {
             cacheCfg.setCacheMode(PARTITIONED);
-            cacheCfg.setNearConfiguration(null);
 
             cacheCfg.setBackups(0);
             cacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));


[07/29] incubator-ignite git commit: # ignite-709_3 do not create system caches on client nodes

Posted by sb...@apache.org.
# ignite-709_3 do not create system caches on client nodes


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

Branch: refs/heads/ignite-709_2
Commit: 8452c8bc09a93b94c1729688b4f2437b99657069
Parents: 07e2955
Author: sboikov <sb...@gridgain.com>
Authored: Thu May 14 17:03:34 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu May 14 17:03:34 2015 +0300

----------------------------------------------------------------------
 .../optimized/OptimizedMarshallerNodeFailoverTest.java           | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8452c8bc/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerNodeFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerNodeFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerNodeFailoverTest.java
index 07fd9e3..35abf7e 100644
--- a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerNodeFailoverTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerNodeFailoverTest.java
@@ -113,7 +113,7 @@ public class OptimizedMarshallerNodeFailoverTest extends GridCommonAbstractTest
 
         cache = true;
 
-        Ignite ignite = startGrid(2); // Check can start on more cache node.
+        Ignite ignite = startGrid(2); // Check can start one more cache node.
 
         assertNotNull(ignite.cache(null));
     }
@@ -122,7 +122,7 @@ public class OptimizedMarshallerNodeFailoverTest extends GridCommonAbstractTest
      * @throws Exception If failed.
      */
     public void testRestartAllNodes() throws Exception {
-        cache = false;
+        cache = true;
 
         String home = U.getIgniteHome();
 


[06/29] incubator-ignite git commit: # ignite-709_3 do not create system caches on client nodes

Posted by sb...@apache.org.
# ignite-709_3 do not create system caches on client nodes


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

Branch: refs/heads/ignite-709_2
Commit: 07e29550b3ff0120d3cd536d05972062ad6f7482
Parents: d02cfe3
Author: sboikov <sb...@gridgain.com>
Authored: Thu May 14 15:45:52 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu May 14 15:45:52 2015 +0300

----------------------------------------------------------------------
 .../IgniteClientDataStructuresAbstractTest.java | 45 ++++++++++++++++++++
 1 file changed, 45 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/07e29550/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
index db16062..8aef0a6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
@@ -230,6 +230,51 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testQueue() throws Exception {
+        Ignite clientNode = clientIgnite();
+
+        final Ignite srvNode = serverNode();
+
+        CollectionConfiguration colCfg = new CollectionConfiguration();
+
+        assertNull(clientNode.queue("q1", 0, null));
+
+        try (IgniteQueue<Integer> queue = clientNode.queue("q1", 0, colCfg)) {
+            assertNotNull(queue);
+
+            queue.add(1);
+
+            assertEquals(1, queue.poll().intValue());
+
+            IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    U.sleep(1000);
+
+                    IgniteQueue<Integer> queue0 = srvNode.queue("q1", 0, null);
+
+                    assertEquals(0, queue0.size());
+
+                    log.info("Add in queue.");
+
+                    queue0.add(2);
+
+                    return null;
+                }
+            });
+
+            log.info("Try take.");
+
+            assertEquals(2, queue.take().intValue());
+
+            log.info("Finished take.");
+
+            fut.get();
+        }
+    }
+
+    /**
      * @return Client node.
      */
     private Ignite clientIgnite() {


[02/29] incubator-ignite git commit: # ignite-709_3 do not create system caches on client nodes

Posted by sb...@apache.org.
# ignite-709_3 do not create system caches on client nodes


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

Branch: refs/heads/ignite-709_2
Commit: 22ec4aae19ee53fb05c14187d0fd62e66490bffa
Parents: 63261f5
Author: sboikov <sb...@gridgain.com>
Authored: Thu May 14 14:32:52 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu May 14 14:32:52 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    | 18 +++-
 .../cache/IgniteSystemCacheOnClientTest.java    | 97 ++++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite4.java       |  1 +
 3 files changed, 115 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/22ec4aae/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 86d80c1..e63dd8b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -736,7 +736,13 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (marshallerCache() == null) {
             assert ctx.config().isClientMode() : "Marshaller cache is missed on server node.";
 
-            IgniteInternalFuture<?> fut = startCacheAsync(CU.MARSH_CACHE_NAME, true);
+            DynamicCacheDescriptor desc = registeredCaches.get(CU.MARSH_CACHE_NAME);
+
+            assert desc != null && desc.cacheConfiguration() != null && desc.cacheType().equals(CacheType.MARSHALLER);
+
+            // On client node user near-only marshaller cache.
+            IgniteInternalFuture<?> fut = dynamicStartCache(desc.cacheConfiguration(),
+                CU.MARSH_CACHE_NAME, new NearCacheConfiguration(), desc.cacheType(), false);
 
             assert fut != null;
 
@@ -2542,6 +2548,16 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @return Utility cache start future.
+     */
+    public IgniteInternalFuture<?> utilityCacheAsync() {
+        if (internalCache(CU.UTILITY_CACHE_NAME) != null)
+            return new GridFinishedFuture<>();
+
+        return startCacheAsync(CU.UTILITY_CACHE_NAME, true);
+    }
+
+    /**
      * @param name Cache name.
      * @param <K> type of keys.
      * @param <V> type of values.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/22ec4aae/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteSystemCacheOnClientTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteSystemCacheOnClientTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteSystemCacheOnClientTest.java
new file mode 100644
index 0000000..52ef7e2
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteSystemCacheOnClientTest.java
@@ -0,0 +1,97 @@
+/*
+ * 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.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+public class IgniteSystemCacheOnClientTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        if (gridName.equals(getTestGridName(1)))
+            cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSystemCacheOnClientNode() throws Exception {
+        startGrids(2);
+
+        final IgniteKernal ignite = (IgniteKernal)ignite(1);
+
+        assertTrue(ignite.configuration().isClientMode());
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return ignite.internalCache(CU.MARSH_CACHE_NAME) != null;
+            }
+        }, 5000);
+
+        GridCacheAdapter marshCache = ignite.internalCache(CU.MARSH_CACHE_NAME);
+
+        assertNotNull(marshCache);
+
+        assertTrue("Marshaller cache on client should have near cache", marshCache.context().isNear());
+
+        marshCache = ((IgniteKernal)ignite(0)).internalCache(CU.MARSH_CACHE_NAME);
+
+        assertFalse(marshCache.context().isNear());
+
+        Collection<ClusterNode> affNodes = marshCache.affinity().mapKeyToPrimaryAndBackups(1);
+
+        assertEquals(1, affNodes.size());
+        assertTrue(affNodes.contains(ignite(0).cluster().localNode()));
+
+        GridCacheAdapter utilityCache = ((IgniteKernal)ignite(0)).internalCache(CU.UTILITY_CACHE_NAME);
+
+        affNodes = utilityCache.affinity().mapKeyToPrimaryAndBackups(1);
+
+        assertEquals(1, affNodes.size());
+        assertTrue(affNodes.contains(ignite(0).cluster().localNode()));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/22ec4aae/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index f7272d4..39f331c 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -125,6 +125,7 @@ public class IgniteCacheTestSuite4 extends TestSuite {
         suite.addTestSuite(IgniteExchangeFutureHistoryTest.class);
 
         suite.addTestSuite(CacheNoValueClassOnServerNodeTest.class);
+        suite.addTestSuite(IgniteSystemCacheOnClientTest.class);
 
         return suite;
     }


[05/29] incubator-ignite git commit: # ignite-709_3 do not create system caches on client nodes

Posted by sb...@apache.org.
# ignite-709_3 do not create system caches on client nodes


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

Branch: refs/heads/ignite-709_2
Commit: d02cfe3357e6ab62bdfebcedb026beb6604da103
Parents: 7ed0ddf
Author: sboikov <sb...@gridgain.com>
Authored: Thu May 14 15:22:12 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu May 14 15:22:12 2015 +0300

----------------------------------------------------------------------
 .../distributed/near/GridNearAtomicCache.java   |  5 ++
 .../GridCacheClientModesAbstractSelfTest.java   | 63 +++++++++++++++++++-
 2 files changed, 67 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d02cfe33/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
index ba3357d..041f83a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
@@ -433,6 +433,11 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
+    @Nullable @Override public V tryPutIfAbsent(K key, V val) throws IgniteCheckedException {
+        return dht.tryPutIfAbsent(key, val);
+    }
+
+    /** {@inheritDoc} */
     @Override public V getAndReplace(K key, V val) throws IgniteCheckedException {
         return dht.getAndReplace(key, val);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d02cfe33/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesAbstractSelfTest.java
index c28ee9b..a665601 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesAbstractSelfTest.java
@@ -24,6 +24,7 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.typedef.*;
 
+import java.io.*;
 import java.util.concurrent.atomic.*;
 
 import static org.apache.ignite.cache.CacheMode.*;
@@ -110,7 +111,7 @@ public abstract class GridCacheClientModesAbstractSelfTest extends GridCacheAbst
         nearOnly.putAll(F.asMap(5, 5, 6, 6, 7, 7, 8, 8, 9, 9));
 
         for (int key = 0; key < 10; key++) {
-            for (int i = 1; i < gridCount(); i++) {
+            for (int i = 0; i < gridCount(); i++) {
                 if (grid(i).affinity(null).isPrimaryOrBackup(grid(i).localNode(), key))
                     assertEquals(key, grid(i).cache(null).localPeek(key, CachePeekMode.ONHEAP));
             }
@@ -120,6 +121,24 @@ public abstract class GridCacheClientModesAbstractSelfTest extends GridCacheAbst
 
             assertNull(nearOnly.localPeek(key, CachePeekMode.PRIMARY, CachePeekMode.BACKUP));
         }
+
+        Integer key = 1000;
+
+        nearOnly.put(key, new TestClass1(key));
+
+        if (nearEnabled())
+            assertNotNull(nearOnly.localPeek(key, CachePeekMode.ALL));
+        else
+            assertNull(nearOnly.localPeek(key, CachePeekMode.ALL));
+
+        for (int i = 0; i < gridCount(); i++) {
+            if (grid(i).affinity(null).isPrimaryOrBackup(grid(i).localNode(), key)) {
+                TestClass1 val = (TestClass1)grid(i).cache(null).localPeek(key, CachePeekMode.ONHEAP);
+
+                assertNotNull(val);
+                assertEquals(key.intValue(), val.val);
+            }
+        }
     }
 
     /**
@@ -147,6 +166,18 @@ public abstract class GridCacheClientModesAbstractSelfTest extends GridCacheAbst
             if (nearEnabled())
                 assertEquals(key, nearOnly.localPeek(key, CachePeekMode.ONHEAP));
         }
+
+        Integer key = 2000;
+
+        dht.put(key, new TestClass2(key));
+
+        TestClass2 val = (TestClass2)nearOnly.get(key);
+
+        assertNotNull(val);
+        assertEquals(key.intValue(), val.val);
+
+        if (nearEnabled())
+            assertNotNull(nearOnly.localPeek(key, CachePeekMode.ONHEAP));
     }
 
     /**
@@ -210,4 +241,34 @@ public abstract class GridCacheClientModesAbstractSelfTest extends GridCacheAbst
 
         return null;
     }
+
+    /**
+     *
+     */
+    static class TestClass1 implements Serializable {
+        /** */
+        int val;
+
+        /**
+         * @param val Value.
+         */
+        public TestClass1(int val) {
+            this.val = val;
+        }
+    }
+
+    /**
+     *
+     */
+    static class TestClass2 implements Serializable {
+        /** */
+        int val;
+
+        /**
+         * @param val Value.
+         */
+        public TestClass2(int val) {
+            this.val = val;
+        }
+    }
 }


[22/29] incubator-ignite git commit: # ignite-709_3

Posted by sb...@apache.org.
# ignite-709_3


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

Branch: refs/heads/ignite-709_2
Commit: ac72c09a4af5358bf16e3d2dd93325c1230c5ebe
Parents: fcaf3f5
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 15 19:02:02 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 15 19:02:02 2015 +0300

----------------------------------------------------------------------
 .../near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java            | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac72c09a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
index 337a921..e78b782 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
@@ -83,6 +83,8 @@ public class GridCacheNearOnlyMultiNodeFullApiSelfTest extends GridCachePartitio
             info("Use grid '" + gridName + "' as near-only.");
 
             cfg.setClientMode(true);
+
+            cfg.setCacheConfiguration();
         }
 
         return cfg;


[26/29] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-709_2' into ignite-709_3

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-709_2' into ignite-709_3


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

Branch: refs/heads/ignite-709_2
Commit: c9d05d706ec60ee06b6cb9bad8e8cbf7b9af533e
Parents: 1d8ae16 6bd3bff
Author: sboikov <sb...@gridgain.com>
Authored: Mon May 18 09:14:36 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon May 18 09:14:36 2015 +0300

----------------------------------------------------------------------
 .../streaming/StreamTransformerExample.java     |   4 +-
 .../streaming/StreamVisitorExample.java         |   4 +-
 .../ignite/examples/streaming/package-info.java |   1 -
 .../streaming/wordcount/CacheConfig.java        |   2 +-
 .../streaming/wordcount/QueryWords.java         |  12 +-
 .../streaming/wordcount/StreamWords.java        |  12 +-
 .../streaming/wordcount/package-info.java       |   1 -
 .../socket/WordsSocketStreamerClient.java       |  82 +++++
 .../socket/WordsSocketStreamerServer.java       | 124 ++++++++
 .../wordcount/socket/package-info.java          |  21 ++
 .../org/apache/ignite/internal/IgnitionEx.java  | 136 +++-----
 .../internal/interop/InteropBootstrap.java      |  34 ++
 .../interop/InteropBootstrapFactory.java        |  39 +++
 .../internal/interop/InteropIgnition.java       | 103 ++++++
 .../internal/interop/InteropProcessor.java      |  25 ++
 .../processors/cache/GridCacheAdapter.java      |   8 +-
 .../processors/cache/GridCacheMapEntry.java     |  35 +--
 .../distributed/GridDistributedLockRequest.java | 111 ++-----
 .../GridDistributedTxFinishRequest.java         |  70 +----
 .../GridDistributedTxPrepareRequest.java        | 112 ++-----
 .../GridDistributedTxRemoteAdapter.java         |  20 +-
 .../distributed/dht/GridDhtLockFuture.java      |   2 -
 .../distributed/dht/GridDhtLockRequest.java     |  45 ++-
 .../dht/GridDhtTransactionalCacheAdapter.java   |   6 -
 .../distributed/dht/GridDhtTxFinishFuture.java  |   3 -
 .../distributed/dht/GridDhtTxFinishRequest.java |  43 ++-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   6 -
 .../distributed/dht/GridDhtTxLocalAdapter.java  |  68 +---
 .../distributed/dht/GridDhtTxPrepareFuture.java |  18 +-
 .../dht/GridDhtTxPrepareRequest.java            |  60 ++--
 .../cache/distributed/dht/GridDhtTxRemote.java  |   8 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   6 -
 .../distributed/near/GridNearLockFuture.java    |   6 -
 .../distributed/near/GridNearLockRequest.java   |  61 ++--
 .../near/GridNearOptimisticTxPrepareFuture.java |  15 +-
 .../GridNearPessimisticTxPrepareFuture.java     |   2 -
 .../near/GridNearTransactionalCache.java        |   4 -
 .../near/GridNearTxFinishRequest.java           |  28 +-
 .../cache/distributed/near/GridNearTxLocal.java |  20 +-
 .../near/GridNearTxPrepareRequest.java          |  52 ++-
 .../distributed/near/GridNearTxRemote.java      |  24 +-
 .../cache/transactions/IgniteInternalTx.java    |  10 -
 .../transactions/IgniteTransactionsImpl.java    |   4 +-
 .../cache/transactions/IgniteTxAdapter.java     |  72 +----
 .../cache/transactions/IgniteTxEntry.java       |  48 +--
 .../cache/transactions/IgniteTxHandler.java     |   6 -
 .../transactions/IgniteTxLocalAdapter.java      | 165 +---------
 .../cache/transactions/IgniteTxLocalEx.java     |  21 +-
 .../cache/transactions/IgniteTxManager.java     |  62 +---
 .../internal/util/nio/GridBufferedParser.java   |   4 -
 .../internal/util/nio/GridDelimitedParser.java  |  91 ++++++
 .../util/nio/GridNioDelimitedBuffer.java        | 106 +++++++
 .../communication/tcp/TcpCommunicationSpi.java  |   2 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  26 --
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |   6 +-
 .../org/apache/ignite/stream/StreamAdapter.java | 111 +++++++
 .../ignite/stream/StreamTupleExtractor.java     |  33 ++
 .../stream/socket/SocketMessageConverter.java   |  31 ++
 .../ignite/stream/socket/SocketStreamer.java    | 218 +++++++++++++
 .../ignite/stream/socket/package-info.java      |  21 ++
 .../near/IgniteCacheNearOnlyTxTest.java         | 190 +++++++++++
 .../util/nio/GridNioDelimitedBufferTest.java    | 112 +++++++
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |   2 +
 .../stream/socket/SocketStreamerSelfTest.java   | 315 +++++++++++++++++++
 .../ignite/stream/socket/package-info.java      |  21 ++
 .../testsuites/IgniteStreamTestSuite.java       |  39 +++
 .../testsuites/IgniteUtilSelfTestSuite.java     |   1 +
 .../processors/cache/jta/CacheJtaManager.java   |   4 +-
 pom.xml                                         |   2 +
 69 files changed, 2022 insertions(+), 1134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9d05d70/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------


[11/29] incubator-ignite git commit: # ignite-709_3

Posted by sb...@apache.org.
# ignite-709_3


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

Branch: refs/heads/ignite-709_2
Commit: 4717ea97ee36913901cbbc49725e930007280966
Parents: 64ed3f1
Author: sboikov <se...@inria.fr>
Authored: Thu May 14 23:34:54 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Thu May 14 23:34:54 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |  26 +--
 .../org/apache/ignite/internal/IgnitionEx.java  |   6 +-
 .../discovery/GridDiscoveryManager.java         |   6 +-
 .../processors/cache/GridCacheProcessor.java    |  23 ++-
 .../processors/cache/GridCacheUtils.java        |  11 ++
 .../affinity/IgniteClientNodeAffinityTest.java  | 185 +++++++++++++++++++
 .../GridCacheQueueApiSelfAbstractTest.java      |   4 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 8 files changed, 224 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4717ea97/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index df6b2ee..5a03ed8 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -145,9 +145,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** Default value for 'readFromBackup' flag. */
     public static final boolean DFLT_READ_FROM_BACKUP = true;
 
-    /** Filter that accepts only server nodes. */
-    public static final IgnitePredicate<ClusterNode> SERVER_NODES = new IgniteServerNodePredicate();
-
     /** Filter that accepts all nodes. */
     public static final IgnitePredicate<ClusterNode> ALL_NODES = new IgniteAllNodesPredicate();
 
@@ -1740,37 +1737,18 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     }
 
     /**
-     * Filter that accepts only server nodes.
-     */
-    public static class IgniteServerNodePredicate implements IgnitePredicate<ClusterNode> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        @Override public boolean apply(ClusterNode n) {
-            Boolean attr = n.attribute(IgniteNodeAttributes.ATTR_CLIENT_MODE);
-
-            return attr != null && !attr;
-        }
-
-        @Override public boolean equals(Object obj) {
-            if (obj == null)
-                return false;
-
-            return obj.getClass().equals(this.getClass());
-        }
-    }
-
-    /**
      *  Filter that accepts all nodes.
      */
     public static class IgniteAllNodesPredicate  implements IgnitePredicate<ClusterNode> {
         /** */
         private static final long serialVersionUID = 0L;
 
+        /** {@inheritDoc} */
         @Override public boolean apply(ClusterNode clusterNode) {
             return true;
         }
 
+        /** {@inheritDoc} */
         @Override public boolean equals(Object obj) {
             if (obj == null)
                 return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4717ea97/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 9c7340e..d7f785f 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
@@ -2015,7 +2015,7 @@ public class IgnitionEx {
             cache.setRebalanceMode(SYNC);
             cache.setWriteSynchronizationMode(FULL_SYNC);
             cache.setAffinity(new RendezvousAffinityFunction(false, 20));
-            cache.setNodeFilter(CacheConfiguration.SERVER_NODES);
+            cache.setNodeFilter(CacheConfiguration.ALL_NODES);
             cache.setStartSize(300);
 
             return cache;
@@ -2036,7 +2036,7 @@ public class IgnitionEx {
             cache.setRebalanceMode(SYNC);
             cache.setWriteSynchronizationMode(FULL_SYNC);
             cache.setAffinity(new RendezvousAffinityFunction(false, 100));
-            cache.setNodeFilter(CacheConfiguration.SERVER_NODES);
+            cache.setNodeFilter(CacheConfiguration.ALL_NODES);
 
             return cache;
         }
@@ -2056,7 +2056,7 @@ public class IgnitionEx {
             ccfg.setRebalanceMode(SYNC);
             ccfg.setWriteSynchronizationMode(FULL_SYNC);
             ccfg.setCacheMode(cfg.getCacheMode());
-            ccfg.setNodeFilter(CacheConfiguration.SERVER_NODES);
+            ccfg.setNodeFilter(CacheConfiguration.ALL_NODES);
             ccfg.setNearConfiguration(new NearCacheConfiguration());
 
             if (cfg.getCacheMode() == PARTITIONED)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4717ea97/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 0950774..59240b8 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
@@ -2515,7 +2515,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
          * @return {@code True} if this node is a data node for given cache.
          */
         public boolean dataNode(ClusterNode node) {
-            return !node.isDaemon() && cacheFilter.apply(node);
+            return !node.isDaemon() && !CU.clientModeNode(node) && cacheFilter.apply(node);
         }
 
         /**
@@ -2523,8 +2523,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
          * @return {@code True} if cache is accessible on the given node.
          */
         public boolean cacheNode(ClusterNode node) {
-            return !node.isClient() && !node.isDaemon() &&
-                (cacheFilter.apply(node) || clientNodes.containsKey(node.id()));
+            return !node.isDaemon() &&
+                ((!CU.clientModeNode(node) && cacheFilter.apply(node)) || clientNodes.containsKey(node.id()));
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4717ea97/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 331e454..6ac5afc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -156,7 +156,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             cfg.setMemoryMode(DFLT_MEMORY_MODE);
 
         if (cfg.getNodeFilter() == null)
-            cfg.setNodeFilter(CacheConfiguration.SERVER_NODES);
+            cfg.setNodeFilter(CacheConfiguration.ALL_NODES);
 
         if (cfg.getAffinity() == null) {
             if (cfg.getCacheMode() == PARTITIONED) {
@@ -713,7 +713,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                 IgnitePredicate filter = ccfg.getNodeFilter();
 
-                if (filter.apply(locNode)) {
+                if (!CU.clientModeNode(locNode) && filter.apply(locNode)) {
                     CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
 
                     CachePluginManager pluginMgr = desc.pluginManager();
@@ -1522,7 +1522,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         ClusterNode locNode = ctx.discovery().localNode();
 
-        boolean affNodeStart = !clientStartOnly && nodeFilter.apply(locNode);
+        boolean affNodeStart = !clientStartOnly && !CU.clientModeNode(locNode) && nodeFilter.apply(locNode);
         boolean clientNodeStart = locNode.id().equals(initiatingNodeId);
 
         if (sharedCtx.cacheContext(CU.cacheId(cfg.getName())) != null)
@@ -1966,7 +1966,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                     // Check if we were asked to start a near cache.
                     if (nearCfg != null) {
-                        if (descCfg.getNodeFilter().apply(ctx.discovery().localNode())) {
+                        ClusterNode locNode = ctx.discovery().localNode();
+
+                        if (!CU.clientModeNode(locNode) && descCfg.getNodeFilter().apply(locNode)) {
                             // If we are on a data node and near cache was enabled, return success, else - fail.
                             if (descCfg.getNearConfiguration() != null)
                                 return new GridFinishedFuture<>();
@@ -2013,7 +2015,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 return new GridFinishedFuture<>(new CacheExistsException("Failed to start near cache " +
                     "(a cache with the given name is not started): " + cacheName));
 
-            if (ccfg.getNodeFilter().apply(ctx.discovery().localNode())) {
+            ClusterNode locNode = ctx.discovery().localNode();
+
+            if (!CU.clientModeNode(locNode) && ccfg.getNodeFilter().apply(locNode)) {
                 if (ccfg.getNearConfiguration() != null)
                     return new GridFinishedFuture<>();
                 else
@@ -2326,8 +2330,13 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "cachePreloadMode",
                 "Cache preload mode", locAttr.cacheRebalanceMode(), rmtAttr.cacheRebalanceMode(), true);
 
-            if (locCfg.getAtomicityMode() == TRANSACTIONAL ||
-                (rmtCfg.getNodeFilter().apply(rmtNode) && locCfg.getNodeFilter().apply(locNode)))
+            boolean checkStore = locCfg.getAtomicityMode() == TRANSACTIONAL ||
+                (!CU.clientModeNode(rmtNode) &&
+                !CU.clientModeNode(locNode) &&
+                rmtCfg.getNodeFilter().apply(rmtNode) &&
+                locCfg.getNodeFilter().apply(locNode));
+
+            if (checkStore)
                 CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "storeFactory", "Store factory",
                     locAttr.storeFactoryClassName(), rmtAttr.storeFactoryClassName(), true);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4717ea97/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 e502b54..163e09a 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
@@ -1799,4 +1799,15 @@ public class GridCacheUtils {
 
         return res;
     }
+    /**
+     * @param node Node.
+     * @return {@code True} if flag {@link IgniteConfiguration#isClientMode()} is set given node.
+     */
+    public static boolean clientModeNode(ClusterNode node) {
+        Boolean clientModeAttr = node.attribute(IgniteNodeAttributes.ATTR_CLIENT_MODE);
+
+        assert clientModeAttr != null : node;
+
+        return clientModeAttr != null && clientModeAttr;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4717ea97/modules/core/src/test/java/org/apache/ignite/cache/affinity/IgniteClientNodeAffinityTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/IgniteClientNodeAffinityTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/IgniteClientNodeAffinityTest.java
new file mode 100644
index 0000000..f0af2c1
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/IgniteClientNodeAffinityTest.java
@@ -0,0 +1,185 @@
+/*
+ * 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.cache.affinity;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.affinity.fair.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.*;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ *
+ */
+public class IgniteClientNodeAffinityTest extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int NODE_CNT = 4;
+
+    /** */
+    private static final String CACHE1 = "cache1";
+
+    /** */
+    private static final String CACHE2 = "cache2";
+
+    /** */
+    private static final String CACHE3 = "cache3";
+
+    /** */
+    private static final String CACHE4 = "cache4";
+
+    /** */
+    private static final String CACHE5 = "cache5";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        if (gridName.equals(getTestGridName(NODE_CNT - 1)))
+            cfg.setClientMode(true);
+
+        CacheConfiguration ccfg1 = new CacheConfiguration();
+
+        ccfg1.setBackups(1);
+        ccfg1.setName(CACHE1);
+        ccfg1.setAffinity(new RendezvousAffinityFunction());
+        ccfg1.setNodeFilter(new TestNodesFilter());
+
+        CacheConfiguration ccfg2 = new CacheConfiguration();
+
+        ccfg2.setBackups(1);
+        ccfg2.setName(CACHE2);
+        ccfg2.setAffinity(new RendezvousAffinityFunction());
+
+        CacheConfiguration ccfg3 = new CacheConfiguration();
+
+        ccfg3.setBackups(1);
+        ccfg3.setName(CACHE3);
+        ccfg3.setAffinity(new FairAffinityFunction());
+        ccfg3.setNodeFilter(new TestNodesFilter());
+
+        CacheConfiguration ccfg4 = new CacheConfiguration();
+
+        ccfg4.setCacheMode(REPLICATED);
+        ccfg4.setName(CACHE4);
+        ccfg4.setNodeFilter(new TestNodesFilter());
+
+        cfg.setCacheConfiguration(ccfg1, ccfg2, ccfg3, ccfg4);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrids(NODE_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientNodeNotInAffinity() throws Exception {
+        checkCache(CACHE1, 2);
+
+        checkCache(CACHE2, 2);
+
+        checkCache(CACHE3, 2);
+
+        checkCache(CACHE4, 3);
+
+        Ignite client = ignite(NODE_CNT - 1);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setBackups(0);
+
+        ccfg.setNodeFilter(new TestNodesFilter());
+
+        try (IgniteCache<Integer, Integer> cache = client.createCache(ccfg)) {
+            checkCache(null, 1);
+        }
+
+        try (IgniteCache<Integer, Integer> cache = client.createCache(ccfg, new NearCacheConfiguration())) {
+            checkCache(null, 1);
+        }
+    }
+
+    /**
+     * @param cacheName Cache name.
+     * @param expNodes Expected number of nodes per partition.
+     */
+    private void checkCache(String cacheName, int expNodes) {
+        log.info("Test cache: " + cacheName);
+
+        Ignite client = ignite(NODE_CNT - 1);
+
+        assertTrue(client.configuration().isClientMode());
+
+        ClusterNode clientNode = client.cluster().localNode();
+
+        for (int i = 0; i < NODE_CNT; i++) {
+            Ignite ignite = ignite(i);
+
+            Affinity<Integer> aff = ignite.affinity(cacheName);
+
+            for (int part = 0; part < aff.partitions(); part++) {
+                Collection<ClusterNode> nodes = aff.mapPartitionToPrimaryAndBackups(part);
+
+                assertEquals(expNodes, nodes.size());
+
+                assertFalse(nodes.contains(clientNode));
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestNodesFilter implements IgnitePredicate<ClusterNode> {
+        /** {@inheritDoc} */
+        @Override public boolean apply(ClusterNode clusterNode) {
+            Boolean attr = clusterNode.attribute(IgniteNodeAttributes.ATTR_CLIENT_MODE);
+
+            assertNotNull(attr);
+
+            assertFalse(attr);
+
+            return true;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4717ea97/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java
index 12397b2..7af0490 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java
@@ -549,7 +549,7 @@ public abstract class GridCacheQueueApiSelfAbstractTest extends IgniteCollection
 
         CollectionConfiguration colCfg2 = collectionConfiguration();
 
-        colCfg2.setNodeFilter(CacheConfiguration.SERVER_NODES);
+        colCfg2.setNodeFilter(CacheConfiguration.ALL_NODES);
 
         IgniteQueue queue1 = grid(0).queue("Queue1", 0, colCfg1);
 
@@ -557,7 +557,7 @@ public abstract class GridCacheQueueApiSelfAbstractTest extends IgniteCollection
 
         assertNotSame(getQueueCache(queue1), getQueueCache(queue2));
 
-        colCfg1.setNodeFilter(CacheConfiguration.SERVER_NODES);
+        colCfg1.setNodeFilter(CacheConfiguration.ALL_NODES);
 
         IgniteQueue queue3 = grid(0).queue("Queue3", 0, colCfg1);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4717ea97/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 9fd4e88..fbeee35 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -20,6 +20,7 @@ package org.apache.ignite.testsuites;
 import junit.framework.*;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.affinity.fair.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.cache.store.jdbc.*;
@@ -85,6 +86,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(IgniteFairAffinityDynamicCacheSelfTest.class);
         suite.addTestSuite(GridCacheAffinityBackupsSelfTest.class);
         suite.addTestSuite(IgniteCacheAffinitySelfTest.class);
+        suite.addTestSuite(IgniteClientNodeAffinityTest.class);
 
         // Swap tests.
         suite.addTestSuite(GridCacheSwapPreloadSelfTest.class);


[16/29] incubator-ignite git commit: # ignite-709_3

Posted by sb...@apache.org.
# ignite-709_3


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

Branch: refs/heads/ignite-709_2
Commit: ef6dd411994f2188217afe6e8d25fc8582e5a62e
Parents: 0646259
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 15 16:58:40 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 15 16:58:40 2015 +0300

----------------------------------------------------------------------
 .../dht/preloader/GridDhtPartitionsExchangeFuture.java      | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef6dd411/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 d288c87..828e472 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
@@ -351,6 +351,15 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                 log.debug("Fetched affinity from remote node, initializing affinity assignment [locNodeId=" +
                     cctx.localNodeId() + ", topVer=" + exchId.topologyVersion() + ']');
 
+            if (affAssignment == null) {
+                affAssignment = new ArrayList<>(cacheCtx.affinity().partitions());
+
+                List<ClusterNode> empty = Collections.emptyList();
+
+                for (int i = 0; i < cacheCtx.affinity().partitions(); i++)
+                    affAssignment.add(empty);
+            }
+
             cacheCtx.affinity().initializeAffinity(exchId.topologyVersion(), affAssignment);
         }
     }


[08/29] incubator-ignite git commit: # ignite-709_3 do not create system caches on client nodes

Posted by sb...@apache.org.
# ignite-709_3 do not create system caches on client nodes


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

Branch: refs/heads/ignite-709_2
Commit: db06cd317fa9f30b8f0c74b120059df6790fdba8
Parents: 8452c8b
Author: sboikov <sb...@gridgain.com>
Authored: Thu May 14 17:31:10 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu May 14 17:31:10 2015 +0300

----------------------------------------------------------------------
 .../service/GridServiceProcessor.java           |  4 +-
 .../service/GridServiceClientNodeTest.java      | 81 ++++++++++++++++++++
 .../testsuites/IgniteKernalSelfTestSuite.java   |  1 +
 3 files changed, 84 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/db06cd31/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 22d1ff0..d22c101 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
@@ -128,10 +128,10 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                 ctx.cache().context().deploy().ignoreOwnership(true);
 
             cfgQryId = cache.context().continuousQueries().executeInternalQuery(
-                new DeploymentListener(), null, true, true);
+                new DeploymentListener(), null, !ctx.config().isClientMode(), true);
 
             assignQryId = cache.context().continuousQueries().executeInternalQuery(
-                new AssignmentListener(), null, true, true);
+                new AssignmentListener(), null, !ctx.config().isClientMode(), true);
         }
         finally {
             if (ctx.deploy().enabled())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/db06cd31/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceClientNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceClientNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceClientNodeTest.java
new file mode 100644
index 0000000..404c32b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceClientNodeTest.java
@@ -0,0 +1,81 @@
+/*
+ * 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.service;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.concurrent.*;
+
+/**
+ *
+ */
+public class GridServiceClientNodeTest extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int NODE_CNT = 3;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi) cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        if (gridName.equals(getTestGridName(NODE_CNT - 1)))
+            cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrids(NODE_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeployFromClient() throws Exception {
+        Ignite ignite = ignite(NODE_CNT - 1);
+
+        assertTrue(ignite.configuration().isClientMode());
+
+        String svcName = "testService";
+
+        CountDownLatch latch = new CountDownLatch(1);
+
+        DummyService.exeLatch(svcName, latch);
+
+        ignite.services().deployClusterSingleton(svcName, new DummyService());
+
+        assertTrue(latch.await(5000, TimeUnit.MILLISECONDS));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/db06cd31/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
index ebd3202..20c32b2 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
@@ -76,6 +76,7 @@ public class IgniteKernalSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridServiceProcessorMultiNodeConfigSelfTest.class);
         suite.addTestSuite(GridServiceProcessorProxySelfTest.class);
         suite.addTestSuite(GridServiceReassignmentSelfTest.class);
+        suite.addTestSuite(GridServiceClientNodeTest.class);
 
         return suite;
     }


[15/29] incubator-ignite git commit: # ignite-709_3

Posted by sb...@apache.org.
# ignite-709_3


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

Branch: refs/heads/ignite-709_2
Commit: 06462598ca5eeda3ea91e5ce841d3087daff3cbb
Parents: 4937822
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 15 16:12:00 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 15 16:12:00 2015 +0300

----------------------------------------------------------------------
 ...cheNearOnlyLruNearEvictionPolicySelfTest.java | 19 ++++++++++---------
 1 file changed, 10 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06462598/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java
index 5d4ff85..c401180 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java
@@ -66,17 +66,18 @@ public class GridCacheNearOnlyLruNearEvictionPolicySelfTest extends GridCommonAb
 
         if (cnt == 0)
             c.setClientMode(true);
+        else {
+            CacheConfiguration cc = new CacheConfiguration();
 
-        CacheConfiguration cc = new CacheConfiguration();
+            cc.setAtomicityMode(atomicityMode);
+            cc.setCacheMode(cacheMode);
+            cc.setWriteSynchronizationMode(PRIMARY_SYNC);
+            cc.setRebalanceMode(SYNC);
+            cc.setStartSize(100);
+            cc.setBackups(0);
 
-        cc.setAtomicityMode(atomicityMode);
-        cc.setCacheMode(cacheMode);
-        cc.setWriteSynchronizationMode(PRIMARY_SYNC);
-        cc.setRebalanceMode(SYNC);
-        cc.setStartSize(100);
-        cc.setBackups(0);
-
-        c.setCacheConfiguration(cc);
+            c.setCacheConfiguration(cc);
+        }
 
         TcpDiscoverySpi disco = new TcpDiscoverySpi();
 


[21/29] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-709_2' into ignite-709_3

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-709_2' into ignite-709_3

Conflicts:
	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java


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

Branch: refs/heads/ignite-709_2
Commit: fcaf3f5116a5c713b4bc07d1f8376566e7dc4eb0
Parents: 994cb14 2d90db7
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 15 18:00:31 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 15 18:00:31 2015 +0300

----------------------------------------------------------------------
 .../internal/events/DiscoveryCustomEvent.java   |  18 +-
 .../managers/discovery/CustomEventListener.java |  31 +
 .../discovery/CustomMessageWrapper.java         |  58 ++
 .../discovery/DiscoveryCustomMessage.java       |  43 +
 .../discovery/GridDiscoveryManager.java         |  99 ++-
 .../cache/DynamicCacheChangeBatch.java          |  15 +-
 .../GridCachePartitionExchangeManager.java      |   4 +-
 .../processors/cache/GridCacheProcessor.java    |   9 +-
 .../continuous/GridContinuousMessageType.java   |  12 -
 .../continuous/GridContinuousProcessor.java     | 796 ++++---------------
 .../processors/continuous/StartRequestData.java | 267 +++++++
 .../StartRoutineAckDiscoveryMessage.java        |  71 ++
 .../StartRoutineDiscoveryMessage.java           |  94 +++
 .../StopRoutineAckDiscoveryMessage.java         |  58 ++
 .../continuous/StopRoutineDiscoveryMessage.java |  58 ++
 .../internal/util/future/SettableFuture.java    |  94 ---
 .../ignite/spi/discovery/DiscoverySpi.java      |   7 +-
 .../discovery/DiscoverySpiCustomMessage.java    |  35 +
 .../spi/discovery/DiscoverySpiListener.java     |   5 +-
 .../discovery/tcp/TcpClientDiscoverySpi.java    | 106 ++-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 113 ++-
 .../TcpDiscoveryCustomEventMessage.java         |   9 +-
 ...acheTcpClientDiscoveryMultiThreadedTest.java |   9 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |   6 +-
 .../continuous/GridEventConsumeSelfTest.java    |  36 +-
 ...GridMessagingNoPeerClassLoadingSelfTest.java |   7 +-
 .../ignite/messaging/GridMessagingSelfTest.java |  13 +-
 .../discovery/AbstractDiscoverySelfTest.java    |   8 +-
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |  10 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 30 files changed, 1156 insertions(+), 937 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fcaf3f51/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fcaf3f51/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fcaf3f51/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------


[12/29] incubator-ignite git commit: # ignite-709_3 merge

Posted by sb...@apache.org.
# ignite-709_3 merge


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

Branch: refs/heads/ignite-709_2
Commit: c1913c45175485543d51e167f7b548e52bdbe13d
Parents: 4717ea9 98743ef
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 15 09:18:07 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 15 09:18:07 2015 +0300

----------------------------------------------------------------------
 bin/ignite-schema-import.bat                    |   2 +-
 bin/ignite-schema-import.sh                     |   2 +-
 bin/ignite.bat                                  |   2 +-
 bin/ignite.sh                                   |   2 +-
 bin/ignitevisorcmd.bat                          |   2 +-
 bin/ignitevisorcmd.sh                           |   2 +-
 .../processors/cache/GridCacheAdapter.java      | 119 +++++------
 .../processors/cache/GridCacheContext.java      |   7 +
 .../processors/cache/GridCacheMapEntry.java     |  70 ++++---
 .../GridDistributedCacheAdapter.java            | 210 ++++++++++++-------
 .../distributed/dht/GridDhtCacheAdapter.java    |  16 +-
 .../dht/GridDhtOffHeapCacheEntry.java           |  63 ++++++
 .../dht/GridDhtTransactionalCacheAdapter.java   |   9 +-
 .../distributed/dht/GridNoStorageCacheMap.java  |   4 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   3 +
 .../atomic/GridDhtAtomicOffHeapCacheEntry.java  |  63 ++++++
 .../dht/colocated/GridDhtColocatedCache.java    |   5 +-
 .../GridDhtColocatedOffHeapCacheEntry.java      |  63 ++++++
 .../GridDhtPartitionsExchangeFuture.java        |   7 -
 .../distributed/near/GridNearCacheAdapter.java  |   3 +
 .../near/GridNearOffHeapCacheEntry.java         |  60 ++++++
 .../cache/local/GridLocalCacheEntry.java        |  18 ++
 .../processors/resource/GridResourceField.java  |   5 +-
 .../processors/resource/GridResourceIoc.java    |  20 +-
 .../resource/GridResourceProcessor.java         |  16 +-
 .../ignite/internal/util/IgniteUtils.java       |   3 +
 .../util/lang/GridComputeJobWrapper.java        |  96 ---------
 .../discovery/tcp/TcpClientDiscoverySpi.java    | 151 +++++++------
 .../tcp/internal/TcpDiscoveryNodesRing.java     |   2 +-
 .../resources/META-INF/classnames.properties    |   1 -
 .../cache/CacheOffheapMapEntrySelfTest.java     | 168 +++++++++++++++
 .../cache/CacheRemoveAllSelfTest.java           |  81 +++++++
 .../distributed/GridCacheLockAbstractTest.java  |   2 -
 ...achePartitionedNearDisabledLockSelfTest.java |  47 +++++
 .../near/NoneRebalanceModeSelfTest.java         |  67 ++++++
 .../GridCacheReplicatedLockSelfTest.java        |   5 +
 .../GridCacheLocalIsolatedNodesSelfTest.java    |  18 +-
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |  37 ++++
 .../discovery/tcp/TcpDiscoveryRestartTest.java  | 199 ++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite2.java       |   2 +
 .../testsuites/IgniteCacheTestSuite4.java       |   4 +
 .../scalar/tests/ScalarCacheQueriesSpec.scala   | 154 +++++++-------
 .../ignite/scalar/tests/ScalarCacheSpec.scala   |  23 +-
 .../scalar/tests/ScalarConversionsSpec.scala    |  43 ++--
 .../scalar/tests/ScalarProjectionSpec.scala     | 128 ++++++-----
 .../scalar/tests/ScalarReturnableSpec.scala     |  41 ++--
 modules/visor-console/pom.xml                   |   2 +-
 .../ignite/visor/VisorRuntimeBaseSpec.scala     |   2 +-
 .../visor/commands/VisorArgListSpec.scala       |  60 +++---
 .../commands/VisorFileNameCompleterSpec.scala   |  34 +--
 .../commands/ack/VisorAckCommandSpec.scala      |  20 +-
 .../commands/alert/VisorAlertCommandSpec.scala  |  68 +++---
 .../cache/VisorCacheClearCommandSpec.scala      |  48 ++---
 .../commands/cache/VisorCacheCommandSpec.scala  |  66 +++---
 .../config/VisorConfigurationCommandSpec.scala  |   8 +-
 .../cswap/VisorCacheSwapCommandSpec.scala       |  24 +--
 .../deploy/VisorDeployCommandSpec.scala         |  10 +-
 .../disco/VisorDiscoveryCommandSpec.scala       |  46 ++--
 .../events/VisorEventsCommandSpec.scala         |  28 +--
 .../visor/commands/gc/VisorGcCommandSpec.scala  |  30 +--
 .../commands/help/VisorHelpCommandSpec.scala    |  57 ++---
 .../commands/kill/VisorKillCommandSpec.scala    |  58 ++---
 .../commands/log/VisorLogCommandSpec.scala      |  10 +-
 .../commands/mem/VisorMemoryCommandSpec.scala   |  77 +++----
 .../commands/node/VisorNodeCommandSpec.scala    |  22 +-
 .../commands/open/VisorOpenCommandSpec.scala    |  16 +-
 .../commands/ping/VisorPingCommandSpec.scala    |  16 +-
 .../commands/start/VisorStartCommandSpec.scala  | 126 +++++------
 .../commands/tasks/VisorTasksCommandSpec.scala  | 112 +++++-----
 .../commands/top/VisorTopologyCommandSpec.scala |  52 ++---
 .../commands/vvm/VisorVvmCommandSpec.scala      |  30 +--
 71 files changed, 1992 insertions(+), 1075 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1913c45/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1913c45/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index 39f331c,aaf7e5b..b3eb899
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@@ -125,8 -125,11 +125,12 @@@ public class IgniteCacheTestSuite4 exte
          suite.addTestSuite(IgniteExchangeFutureHistoryTest.class);
  
          suite.addTestSuite(CacheNoValueClassOnServerNodeTest.class);
 +        suite.addTestSuite(IgniteSystemCacheOnClientTest.class);
  
+         suite.addTestSuite(CacheRemoveAllSelfTest.class);
+ 
+         suite.addTestSuite(CacheOffheapMapEntrySelfTest.class);
+ 
          return suite;
      }
  }


[27/29] incubator-ignite git commit: # ignite-709_3

Posted by sb...@apache.org.
# ignite-709_3


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

Branch: refs/heads/ignite-709_2
Commit: dc315c0d22ade7a51936e97541896fa0b04760f2
Parents: c9d05d7
Author: sboikov <sb...@gridgain.com>
Authored: Mon May 18 09:21:42 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon May 18 09:21:42 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/MarshallerContextImpl.java    | 2 +-
 .../ignite/internal/processors/cache/GridCacheProcessor.java | 8 ++------
 .../dht/preloader/GridDhtPartitionsExchangeFuture.java       | 1 +
 .../internal/processors/service/GridServiceProcessor.java    | 4 ++--
 4 files changed, 6 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc315c0d/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
index 13160b3..e614408 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
@@ -59,7 +59,7 @@ public class MarshallerContextImpl extends MarshallerContextAdapter {
         ctx.cache().marshallerCache().context().continuousQueries().executeInternalQuery(
             new ContinuousQueryListener(log, workDir),
             null,
-            !ctx.config().isClientMode(),
+            ctx.cache().marshallerCache().context().affinityNode(),
             true
         );
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc315c0d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 923cbbd..fed9504 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -1879,9 +1879,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                     // Check if we were asked to start a near cache.
                     if (nearCfg != null) {
-                        ClusterNode locNode = ctx.discovery().localNode();
-
-                        if (CU.affinityNode(locNode, descCfg.getNodeFilter())) {
+                        if (CU.affinityNode(ctx.discovery().localNode(), descCfg.getNodeFilter())) {
                             // If we are on a data node and near cache was enabled, return success, else - fail.
                             if (descCfg.getNearConfiguration() != null)
                                 return new GridFinishedFuture<>();
@@ -1928,9 +1926,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 return new GridFinishedFuture<>(new CacheExistsException("Failed to start near cache " +
                     "(a cache with the given name is not started): " + cacheName));
 
-            ClusterNode locNode = ctx.discovery().localNode();
-
-            if (CU.affinityNode(locNode, ccfg.getNodeFilter())) {
+            if (CU.affinityNode(ctx.discovery().localNode(), ccfg.getNodeFilter())) {
                 if (ccfg.getNearConfiguration() != null)
                     return new GridFinishedFuture<>();
                 else

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc315c0d/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 828e472..599e391 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
@@ -365,6 +365,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     }
 
     /**
+     * @param cacheCtx Cache context.
      * @return {@code True} if local node can calculate affinity on it's own for this partition map exchange.
      */
     private boolean canCalculateAffinity(GridCacheContext cacheCtx) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc315c0d/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 d22c101..a9a15e9 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
@@ -128,10 +128,10 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                 ctx.cache().context().deploy().ignoreOwnership(true);
 
             cfgQryId = cache.context().continuousQueries().executeInternalQuery(
-                new DeploymentListener(), null, !ctx.config().isClientMode(), true);
+                new DeploymentListener(), null, cache.context().affinityNode(), true);
 
             assignQryId = cache.context().continuousQueries().executeInternalQuery(
-                new AssignmentListener(), null, !ctx.config().isClientMode(), true);
+                new AssignmentListener(), null, cache.context().affinityNode(), true);
         }
         finally {
             if (ctx.deploy().enabled())


[14/29] incubator-ignite git commit: # ignite-709_3

Posted by sb...@apache.org.
# ignite-709_3


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

Branch: refs/heads/ignite-709_2
Commit: 4937822ab437288a189505736d89c1ba83670a97
Parents: 6d11043
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 15 15:32:59 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 15 15:32:59 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/internal/MarshallerContextImpl.java     | 2 +-
 .../cache/datastructures/CacheDataStructuresManager.java           | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4937822a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
index 85939a6..13160b3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
@@ -59,7 +59,7 @@ public class MarshallerContextImpl extends MarshallerContextAdapter {
         ctx.cache().marshallerCache().context().continuousQueries().executeInternalQuery(
             new ContinuousQueryListener(log, workDir),
             null,
-            true,
+            !ctx.config().isClientMode(),
             true
         );
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4937822a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
index fa8d192..b5c5161 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
@@ -218,7 +218,7 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
                         }
                     },
                     new QueueHeaderPredicate(),
-                    cctx.isLocal() || cctx.isReplicated(),
+                    cctx.isLocal() || (cctx.isReplicated() && cctx.affinityNode()),
                     true);
             }
 


[24/29] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-709_2' into ignite-709_3

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-709_2' into ignite-709_3


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

Branch: refs/heads/ignite-709_2
Commit: 46b36e8fd728edb71aee578bbace93b746888bf5
Parents: ce5eb8b aaa7425
Author: sboikov <se...@inria.fr>
Authored: Fri May 15 20:50:29 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Fri May 15 20:50:29 2015 +0300

----------------------------------------------------------------------
 .../discovery/tcp/TcpClientDiscoverySpi.java    |  4 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 28 ---------
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   | 27 +++++++++
 .../continuous/GridEventConsumeSelfTest.java    | 63 ++++++++++++--------
 .../testframework/junits/GridAbstractTest.java  | 16 +++++
 5 files changed, 82 insertions(+), 56 deletions(-)
----------------------------------------------------------------------



[23/29] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-709_2' into ignite-709_3

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-709_2' into ignite-709_3


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

Branch: refs/heads/ignite-709_2
Commit: ce5eb8bd2da97cb071eb0f7743d2224dd2c03ed3
Parents: ac72c09 fd94eab
Author: sboikov <se...@inria.fr>
Authored: Fri May 15 19:26:27 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Fri May 15 19:26:27 2015 +0300

----------------------------------------------------------------------
 .../discovery/tcp/TcpClientDiscoverySpi.java    | 20 ++++-----
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 11 ++++-
 .../TcpDiscoveryNodeAddFinishedMessage.java     | 43 ++++++++++++++++++++
 ...acheTcpClientDiscoveryMultiThreadedTest.java |  2 +-
 .../continuous/GridEventConsumeSelfTest.java    |  4 +-
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 38 ++++++++++++++---
 6 files changed, 99 insertions(+), 19 deletions(-)
----------------------------------------------------------------------



[13/29] incubator-ignite git commit: # ignite-709_3

Posted by sb...@apache.org.
# ignite-709_3


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

Branch: refs/heads/ignite-709_2
Commit: 6d1104372142898f7d46fa72265473d06e1eb959
Parents: c1913c4
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 15 13:37:24 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 15 15:14:10 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/IgniteEx.java    |   3 +-
 .../apache/ignite/internal/IgniteKernal.java    |  24 ++-
 .../discovery/GridDiscoveryManager.java         |  37 ++--
 .../cache/DynamicCacheDescriptor.java           |   2 +
 .../processors/cache/GridCacheProcessor.java    | 183 ++++------------
 .../processors/cache/GridCacheUtils.java        |  12 +-
 .../cache/affinity/GridCacheAffinityImpl.java   |  10 +-
 .../preloader/GridDhtPartitionDemandPool.java   |   4 -
 .../datastructures/DataStructuresProcessor.java |  12 +-
 .../affinity/IgniteClientNodeAffinityTest.java  |   3 -
 .../IgniteDynamicClientCacheStartSelfTest.java  | 213 +++++++++++++++++++
 .../cache/IgniteSystemCacheOnClientTest.java    |   2 +-
 .../testsuites/IgniteCacheTestSuite4.java       |   1 +
 13 files changed, 313 insertions(+), 193 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6d110437/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
index bc7e722..4845d51 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
@@ -35,9 +35,8 @@ public interface IgniteEx extends Ignite {
      * Gets utility cache.
      *
      * @return Utility cache.
-     * @throws IgniteCheckedException If failed.
      */
-    public <K extends GridCacheUtilityKey, V> IgniteInternalCache<K, V> utilityCache() throws IgniteCheckedException;
+    public <K extends GridCacheUtilityKey, V> IgniteInternalCache<K, V> utilityCache();
 
     /**
      * Gets the cache instance for the given name if one is configured or

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6d110437/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 1c68a82..49b5f22 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
@@ -2362,7 +2362,11 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         try {
             ctx.cache().dynamicStartCache(null, cacheName, nearCfg, true).get();
 
-            return ctx.cache().publicJCache(cacheName);
+            IgniteCacheProxy<K, V> cache = ctx.cache().publicJCache(cacheName);
+
+            checkNearCacheStarted(cache);
+
+            return cache;
         }
         catch (IgniteCheckedException e) {
             throw CU.convertToCacheException(e);
@@ -2382,7 +2386,11 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         try {
             ctx.cache().dynamicStartCache(null, cacheName, nearCfg, false).get();
 
-            return ctx.cache().publicJCache(cacheName);
+            IgniteCacheProxy<K, V> cache = ctx.cache().publicJCache(cacheName);
+
+            checkNearCacheStarted(cache);
+
+            return cache;
         }
         catch (IgniteCheckedException e) {
             throw CU.convertToCacheException(e);
@@ -2392,6 +2400,15 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         }
     }
 
+    /**
+     * @param cache Cache.
+     */
+    private void checkNearCacheStarted(IgniteCacheProxy<?, ?> cache) {
+        if (!cache.context().isNear())
+            throw new IgniteException("Failed to start near cache " +
+                "(a cache with the same name without near cache is already started)");
+    }
+
     /** {@inheritDoc} */
     @Override public void destroyCache(String cacheName) {
         guard();
@@ -2462,8 +2479,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Override public <K extends GridCacheUtilityKey, V> IgniteInternalCache<K, V> utilityCache()
-        throws IgniteCheckedException {
+    @Override public <K extends GridCacheUtilityKey, V> IgniteInternalCache<K, V> utilityCache() {
         guard();
 
         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6d110437/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 59240b8..eac96b2 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
@@ -214,6 +214,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      *
      * @param cacheName Cache name.
      * @param filter Cache filter.
+     * @param nearEnabled Near enabled flag.
      * @param loc {@code True} if cache is local.
      */
     public void setCacheFilter(
@@ -240,12 +241,13 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      *
      * @param cacheName Cache name.
      * @param clientNodeId Near node ID.
+     * @param nearEnabled Near enabled flag.
      */
     public void addClientNode(String cacheName, UUID clientNodeId, boolean nearEnabled) {
-        CachePredicate predicate = registeredCaches.get(cacheName);
+        CachePredicate pred = registeredCaches.get(cacheName);
 
-        if (predicate != null)
-            predicate.addClientNode(clientNodeId, nearEnabled);
+        if (pred != null)
+            pred.addClientNode(clientNodeId, nearEnabled);
     }
 
     /**
@@ -1256,9 +1258,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return {@code True} if node is a cache data node.
      */
     public boolean cacheAffinityNode(ClusterNode node, String cacheName) {
-        CachePredicate predicate = registeredCaches.get(cacheName);
+        CachePredicate pred = registeredCaches.get(cacheName);
 
-        return predicate != null && predicate.dataNode(node);
+        return pred != null && pred.dataNode(node);
     }
 
     /**
@@ -1267,9 +1269,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return {@code True} if node has near cache enabled.
      */
     public boolean cacheNearNode(ClusterNode node, String cacheName) {
-        CachePredicate predicate = registeredCaches.get(cacheName);
+        CachePredicate pred = registeredCaches.get(cacheName);
 
-        return predicate != null && predicate.nearNode(node);
+        return pred != null && pred.nearNode(node);
     }
 
     /**
@@ -1278,9 +1280,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return {@code True} if node has client cache (without near cache).
      */
     public boolean cacheClientNode(ClusterNode node, String cacheName) {
-        CachePredicate predicate = registeredCaches.get(cacheName);
+        CachePredicate pred = registeredCaches.get(cacheName);
 
-        return predicate != null && predicate.clientNode(node);
+        return pred != null && pred.clientNode(node);
     }
 
     /**
@@ -1289,9 +1291,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return If cache with the given name is accessible on the given node.
      */
     public boolean cacheNode(ClusterNode node, String cacheName) {
-        CachePredicate predicate = registeredCaches.get(cacheName);
+        CachePredicate pred = registeredCaches.get(cacheName);
 
-        return predicate != null && predicate.cacheNode(node);
+        return pred != null && pred.cacheNode(node);
     }
 
     /**
@@ -2480,11 +2482,12 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         private boolean loc;
 
         /** Collection of client near nodes. */
-        private Map<UUID, Boolean> clientNodes;
+        private ConcurrentHashMap<UUID, Boolean> clientNodes;
 
         /**
          * @param cacheFilter Cache filter.
          * @param nearEnabled Near enabled flag.
+         * @param loc {@code True} if cache is local.
          */
         private CachePredicate(IgnitePredicate<ClusterNode> cacheFilter, boolean nearEnabled, boolean loc) {
             assert cacheFilter != null;
@@ -2498,9 +2501,10 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
         /**
          * @param nodeId Near node ID to add.
+         * @param nearEnabled Near enabled flag.
          */
         public void addClientNode(UUID nodeId, boolean nearEnabled) {
-            clientNodes.put(nodeId, nearEnabled);
+            clientNodes.putIfAbsent(nodeId, nearEnabled);
         }
 
         /**
@@ -2515,7 +2519,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
          * @return {@code True} if this node is a data node for given cache.
          */
         public boolean dataNode(ClusterNode node) {
-            return !node.isDaemon() && !CU.clientModeNode(node) && cacheFilter.apply(node);
+            return !node.isDaemon() && CU.affinityNode(node, cacheFilter);
         }
 
         /**
@@ -2523,8 +2527,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
          * @return {@code True} if cache is accessible on the given node.
          */
         public boolean cacheNode(ClusterNode node) {
-            return !node.isDaemon() &&
-                ((!CU.clientModeNode(node) && cacheFilter.apply(node)) || clientNodes.containsKey(node.id()));
+            return !node.isDaemon() && (CU.affinityNode(node, cacheFilter) || clientNodes.containsKey(node.id()));
         }
 
         /**
@@ -2535,7 +2538,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             if (node.isDaemon())
                 return false;
 
-            if (nearEnabled && cacheFilter.apply(node))
+            if (nearEnabled && CU.affinityNode(node, cacheFilter))
                 return true;
 
             Boolean near = clientNodes.get(node.id());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6d110437/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
index 6f6f422..a27ebd4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
@@ -62,6 +62,7 @@ public class DynamicCacheDescriptor {
     private final CachePluginManager pluginMgr;
 
     /**
+     * @param ctx Context.
      * @param cacheCfg Cache configuration.
      * @param cacheType Cache type.
      * @param template {@code True} if this is template configuration.
@@ -76,6 +77,7 @@ public class DynamicCacheDescriptor {
         this.cacheType = cacheType;
         this.template = template;
         this.deploymentId = deploymentId;
+
         pluginMgr = new CachePluginManager(ctx, cacheCfg);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6d110437/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 6ac5afc..9469e65 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -127,9 +127,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** Count down latch for caches. */
     private final CountDownLatch cacheStartedLatch = new CountDownLatch(1);
 
-    /** */
-    private final GridFutureAdapter<Object> sysCacheStartFut = new GridFutureAdapter<>();
-
     /**
      * @param ctx Kernal context.
      */
@@ -628,6 +625,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     cfg.getNearConfiguration() != null,
                     cfg.getCacheMode() == LOCAL);
 
+                ctx.discovery().addClientNode(cfg.getName(),
+                    ctx.localNodeId(),
+                    cfg.getNearConfiguration() != null);
+
                 if (!cacheType.userCache())
                     stopSeq.addLast(cfg.getName());
                 else
@@ -664,9 +665,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void onKernalStart() throws IgniteCheckedException {
-        DynamicCacheDescriptor marshCacheDesc = null;
-        DynamicCacheDescriptor utilityCacheDesc = null;
-        DynamicCacheDescriptor atomicsCacheDesc = null;
+        List<GridCacheAdapter<?, ?>> locCaches = new ArrayList<>(registeredCaches.size());
 
         try {
             if (ctx.config().isDaemon())
@@ -713,7 +712,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                 IgnitePredicate filter = ccfg.getNodeFilter();
 
-                if (!CU.clientModeNode(locNode) && filter.apply(locNode)) {
+                boolean loc = desc.locallyConfigured();
+
+                if (loc || CU.affinityNode(locNode, filter)) {
                     CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
 
                     CachePluginManager pluginMgr = desc.pluginManager();
@@ -733,49 +734,23 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     startCache(cache);
 
                     jCacheProxies.put(maskNull(name), new IgniteCacheProxy(ctx, cache, null, false));
-                }
 
-                if (CU.MARSH_CACHE_NAME.equals(ccfg.getName()))
-                    marshCacheDesc = desc;
-                else if (CU.UTILITY_CACHE_NAME.equals(ccfg.getName()))
-                    utilityCacheDesc = desc;
-                else if (CU.ATOMICS_CACHE_NAME.equals(ccfg.getName()))
-                    atomicsCacheDesc = desc;
+                    if (loc)
+                        locCaches.add(cache);
+                }
             }
         }
         finally {
             cacheStartedLatch.countDown();
         }
 
-        if (ctx.config().isClientMode()) {
-            assert marshCacheDesc != null;
-            assert utilityCacheDesc != null;
-            assert atomicsCacheDesc != null;
-
-            Collection<DynamicCacheChangeRequest> reqs = new ArrayList<>();
-
-            reqs.add(clientSystemCacheRequest(marshCacheDesc, new NearCacheConfiguration()));
-            reqs.add(clientSystemCacheRequest(utilityCacheDesc, null));
-            reqs.add(clientSystemCacheRequest(atomicsCacheDesc, new NearCacheConfiguration()));
-
-            startClientSystemCaches(reqs);
-
-            sysCacheStartFut.listen(new CI1<IgniteInternalFuture<Object>>() {
-                @Override public void apply(IgniteInternalFuture<Object> fut) {
-                    try {
-                        marshallerCacheCallbacks();
-                    }
-                    catch (IgniteCheckedException e) {
-                        U.error(log, "Failed to initialize marshaller context.", e);
-                    }
-                }
-            });
-        }
-        else {
-            sysCacheStartFut.onDone();
+        ctx.marshallerContext().onMarshallerCacheStarted(ctx);
 
-            marshallerCacheCallbacks();
-        }
+        marshallerCache().context().preloader().syncFuture().listen(new CIX1<IgniteInternalFuture<?>>() {
+            @Override public void applyx(IgniteInternalFuture<?> f) throws IgniteCheckedException {
+                ctx.marshallerContext().onMarshallerCachePreloaded(ctx);
+            }
+        });
 
         // Must call onKernalStart on shared managers after creation of fetched caches.
         for (GridCacheSharedManager<?, ?> mgr : sharedCtx.managers())
@@ -811,80 +786,19 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             onKernalStart(cache);
 
         // Wait for caches in SYNC preload mode.
-        for (GridCacheAdapter<?, ?> cache : caches.values()) {
-            if (cache.context().started()) {
-                CacheConfiguration cfg = cache.configuration();
+        for (GridCacheAdapter<?, ?> cache : locCaches) {
+            CacheConfiguration cfg = cache.configuration();
 
-                if (cfg.getRebalanceMode() == SYNC) {
-                    if (cfg.getCacheMode() == REPLICATED ||
-                        (cfg.getCacheMode() == PARTITIONED && cfg.getRebalanceDelay() >= 0))
-                        cache.preloader().syncFuture().get();
-                }
+            if (cfg.getRebalanceMode() == SYNC) {
+                if (cfg.getCacheMode() == REPLICATED ||
+                    (cfg.getCacheMode() == PARTITIONED && cfg.getRebalanceDelay() >= 0))
+                    cache.preloader().syncFuture().get();
             }
         }
 
         ctx.cacheObjects().onCacheProcessorStarted();
     }
 
-    /**
-     * @param reqs Start requests.
-     */
-    private void startClientSystemCaches(Collection<DynamicCacheChangeRequest> reqs) {
-        assert !F.isEmpty(reqs) : reqs;
-
-        GridCompoundFuture<Object, Object> fut = new GridCompoundFuture<>();
-
-        for (DynamicCacheStartFuture startFut : initiateCacheChanges(reqs))
-            fut.add(startFut);
-
-        fut.markInitialized();
-
-        fut.listen(new CI1<IgniteInternalFuture<Object>>() {
-            @Override public void apply(IgniteInternalFuture<Object> fut) {
-                sysCacheStartFut.onDone();
-            }
-        });
-    }
-
-    /**
-     * @param cacheDesc Cache descriptor.
-     * @return Cache change request.
-     */
-    private DynamicCacheChangeRequest clientSystemCacheRequest(
-        DynamicCacheDescriptor cacheDesc,
-        @Nullable NearCacheConfiguration nearCfg)
-    {
-        DynamicCacheChangeRequest desc = new DynamicCacheChangeRequest(
-            cacheDesc.cacheConfiguration().getName(),
-            ctx.localNodeId());
-
-        desc.clientStartOnly(true);
-
-        desc.nearCacheConfiguration(nearCfg);
-
-        desc.deploymentId(cacheDesc.deploymentId());
-
-        desc.startCacheConfiguration(cacheDesc.cacheConfiguration());
-
-        desc.cacheType(cacheDesc.cacheType());
-
-        return desc;
-    }
-
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    private void marshallerCacheCallbacks() throws IgniteCheckedException {
-        ctx.marshallerContext().onMarshallerCacheStarted(ctx);
-
-        marshallerCache().context().preloader().syncFuture().listen(new CIX1<IgniteInternalFuture<?>>() {
-            @Override
-            public void applyx(IgniteInternalFuture<?> f) throws IgniteCheckedException {
-                ctx.marshallerContext().onMarshallerCachePreloaded(ctx);
-            }
-        });
-    }
-
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void stop(boolean cancel) throws IgniteCheckedException {
@@ -928,8 +842,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     @Override public void onKernalStop(boolean cancel) {
         cacheStartedLatch.countDown();
 
-        sysCacheStartFut.onDone();
-
         if (ctx.config().isDaemon())
             return;
 
@@ -1522,7 +1434,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         ClusterNode locNode = ctx.discovery().localNode();
 
-        boolean affNodeStart = !clientStartOnly && !CU.clientModeNode(locNode) && nodeFilter.apply(locNode);
+        boolean affNodeStart = !clientStartOnly && CU.affinityNode(locNode, nodeFilter);
         boolean clientNodeStart = locNode.id().equals(initiatingNodeId);
 
         if (sharedCtx.cacheContext(CU.cacheId(cfg.getName())) != null)
@@ -1968,7 +1880,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     if (nearCfg != null) {
                         ClusterNode locNode = ctx.discovery().localNode();
 
-                        if (!CU.clientModeNode(locNode) && descCfg.getNodeFilter().apply(locNode)) {
+                        if (CU.affinityNode(locNode, descCfg.getNodeFilter())) {
                             // If we are on a data node and near cache was enabled, return success, else - fail.
                             if (descCfg.getNearConfiguration() != null)
                                 return new GridFinishedFuture<>();
@@ -2017,7 +1929,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             ClusterNode locNode = ctx.discovery().localNode();
 
-            if (!CU.clientModeNode(locNode) && ccfg.getNodeFilter().apply(locNode)) {
+            if (CU.affinityNode(locNode, ccfg.getNodeFilter())) {
                 if (ccfg.getNearConfiguration() != null)
                     return new GridFinishedFuture<>();
                 else
@@ -2305,6 +2217,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /**
      * Checks that remote caches has configuration compatible with the local.
      *
+     * @param locCfg Local configuration.
+     * @param rmtCfg Remote configuration.
      * @param rmtNode Remote node.
      * @throws IgniteCheckedException If check failed.
      */
@@ -2330,13 +2244,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "cachePreloadMode",
                 "Cache preload mode", locAttr.cacheRebalanceMode(), rmtAttr.cacheRebalanceMode(), true);
 
-            boolean checkStore = locCfg.getAtomicityMode() == TRANSACTIONAL ||
-                (!CU.clientModeNode(rmtNode) &&
-                !CU.clientModeNode(locNode) &&
-                rmtCfg.getNodeFilter().apply(rmtNode) &&
-                locCfg.getNodeFilter().apply(locNode));
-
-            if (checkStore)
+            if (locCfg.getAtomicityMode() == TRANSACTIONAL ||
+                (CU.affinityNode(rmtNode, rmtCfg.getNodeFilter()) && CU.affinityNode(locNode, locCfg.getNodeFilter())))
                 CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "storeFactory", "Store factory",
                     locAttr.storeFactoryClassName(), rmtAttr.storeFactoryClassName(), true);
 
@@ -2607,26 +2516,18 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * Gets utility cache.
      *
      * @return Utility cache.
-     * @throws IgniteCheckedException If failed.
      */
-    public <K, V> GridCacheAdapter<K, V> utilityCache() throws IgniteCheckedException {
-        GridCacheAdapter<K, V> cache = internalCache(CU.UTILITY_CACHE_NAME);
-
-        if (cache != null)
-            return cache;
-
-        assert ctx.config().isClientMode() : "Utility cache is missed on server node.";
-
-        sysCacheStartFut.get();
-
+    public <K, V> GridCacheAdapter<K, V> utilityCache() {
         return internalCache(CU.UTILITY_CACHE_NAME);
     }
 
     /**
-     * @return Utility cache start future.
+     * Gets utility cache for atomic data structures.
+     *
+     * @return Utility cache for atomic data structures.
      */
-    public IgniteInternalFuture<?> systemCachesStartFuture() {
-        return sysCacheStartFut;
+    public <K, V> IgniteInternalCache<K, V> atomicsCache() {
+        return cache(CU.ATOMICS_CACHE_NAME);
     }
 
     /**
@@ -2663,7 +2564,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @return Cache instance for given name.
      * @throws IgniteCheckedException If failed.
      */
-    public <K, V> IgniteCache<K, V> publicJCache(@Nullable String cacheName) throws IgniteCheckedException {
+    public <K, V> IgniteCacheProxy<K, V> publicJCache(@Nullable String cacheName) throws IgniteCheckedException {
         return publicJCache(cacheName, true);
     }
 
@@ -2677,7 +2578,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings({"unchecked", "ConstantConditions"})
-    @Nullable public <K, V> IgniteCache<K, V> publicJCache(@Nullable String cacheName, boolean failIfNotStarted)
+    @Nullable public <K, V> IgniteCacheProxy<K, V> publicJCache(@Nullable String cacheName, boolean failIfNotStarted)
         throws IgniteCheckedException
     {
         if (log.isDebugEnabled())
@@ -2685,7 +2586,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         String masked = maskNull(cacheName);
 
-        IgniteCache<K,V> cache = (IgniteCache<K, V>)jCacheProxies.get(masked);
+        IgniteCacheProxy<?, ?> cache = jCacheProxies.get(masked);
 
         DynamicCacheDescriptor desc = registeredCaches.get(masked);
 
@@ -2695,7 +2596,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (cache == null)
            cache = startJCache(cacheName, failIfNotStarted);
 
-        return cache;
+        return (IgniteCacheProxy<K, V>)cache;
     }
 
     /**
@@ -2705,7 +2606,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @return Cache instance for given name.
      * @throws IgniteCheckedException If failed.
      */
-    private IgniteCache startJCache(String cacheName, boolean failIfNotStarted) throws IgniteCheckedException {
+    private IgniteCacheProxy startJCache(String cacheName, boolean failIfNotStarted) throws IgniteCheckedException {
         String masked = maskNull(cacheName);
 
         DynamicCacheDescriptor desc = registeredCaches.get(masked);
@@ -2735,7 +2636,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         F.first(initiateCacheChanges(F.asList(req))).get();
 
-        IgniteCache cache = jCacheProxies.get(masked);
+        IgniteCacheProxy cache = jCacheProxies.get(masked);
 
         if (cache == null && failIfNotStarted)
             throw new IllegalArgumentException("Cache is not started: " + cacheName);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6d110437/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 163e09a..ef04ff4 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
@@ -1697,7 +1697,7 @@ public class GridCacheUtils {
     /**
      * @param aff Affinity.
      * @param n Node.
-     * @return Predicate that evaulates to {@code true} if entry is primary for node.
+     * @return Predicate that evaluates to {@code true} if entry is primary for node.
      */
     public static CacheEntryPredicate cachePrimary(
         final Affinity aff,
@@ -1799,15 +1799,19 @@ public class GridCacheUtils {
 
         return res;
     }
+
     /**
      * @param node Node.
-     * @return {@code True} if flag {@link IgniteConfiguration#isClientMode()} is set given node.
+     * @param filter Node filter.
+     * @return {@code True} if node is not client node and pass given filter.
      */
-    public static boolean clientModeNode(ClusterNode node) {
+    public static boolean affinityNode(ClusterNode node, IgnitePredicate<ClusterNode> filter) {
         Boolean clientModeAttr = node.attribute(IgniteNodeAttributes.ATTR_CLIENT_MODE);
 
         assert clientModeAttr != null : node;
 
-        return clientModeAttr != null && clientModeAttr;
+        boolean clientMode = clientModeAttr != null && clientModeAttr;
+
+        return !clientMode && filter.apply(node);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6d110437/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
index 0186a90..0790052 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
@@ -84,9 +84,7 @@ public class GridCacheAffinityImpl<K, V> implements Affinity<K> {
     @Override public int[] primaryPartitions(ClusterNode n) {
         A.notNull(n, "n");
 
-        AffinityTopologyVersion topVer = new AffinityTopologyVersion(cctx.discovery().topologyVersion());
-
-        Set<Integer> parts = cctx.affinity().primaryPartitions(n.id(), topVer);
+        Set<Integer> parts = cctx.affinity().primaryPartitions(n.id(), topologyVersion());
 
         return U.toIntArray(parts);
     }
@@ -95,9 +93,7 @@ public class GridCacheAffinityImpl<K, V> implements Affinity<K> {
     @Override public int[] backupPartitions(ClusterNode n) {
         A.notNull(n, "n");
 
-        AffinityTopologyVersion topVer = new AffinityTopologyVersion(cctx.discovery().topologyVersion());
-
-        Set<Integer> parts = cctx.affinity().backupPartitions(n.id(), topVer);
+        Set<Integer> parts = cctx.affinity().backupPartitions(n.id(), topologyVersion());
 
         return U.toIntArray(parts);
     }
@@ -108,7 +104,7 @@ public class GridCacheAffinityImpl<K, V> implements Affinity<K> {
 
         Collection<Integer> parts = new HashSet<>();
 
-        AffinityTopologyVersion topVer = new AffinityTopologyVersion(cctx.discovery().topologyVersion());
+        AffinityTopologyVersion topVer = topologyVersion();
 
         for (int partsCnt = partitions(), part = 0; part < partsCnt; part++) {
             for (ClusterNode affNode : cctx.affinity().nodes(part, topVer)) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6d110437/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
index 4153c5f..633f237 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
@@ -824,10 +824,6 @@ public class GridDhtPartitionDemandPool<K, V> {
                         log.debug("Waiting for marshaller cache preload [cacheName=" + cctx.name() + ']');
 
                     try {
-                        cctx.kernalContext().cache().systemCachesStartFuture().get();
-
-                        cctx.kernalContext().cache().marshallerCache().context().awaitStarted();
-
                         cctx.kernalContext().cache().marshallerCache().preloader().syncFuture().get();
                     }
                     catch (IgniteInterruptedCheckedException ignored) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6d110437/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index b6d4b40..72911af 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -112,7 +112,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public void onKernalStart() throws IgniteCheckedException {
+    @Override public void onKernalStart() {
         if (ctx.config().isDaemon())
             return;
 
@@ -123,15 +123,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
         assert utilityCache != null;
 
         if (atomicCfg != null) {
-            IgniteInternalCache atomicsCache = ctx.cache().internalCache(CU.ATOMICS_CACHE_NAME);
-
-            if (atomicsCache == null) {
-                assert ctx.config().isClientMode() : "Atomics cache is missed on server node.";
-
-                ctx.cache().systemCachesStartFuture().get();
-
-                atomicsCache = ctx.cache().internalCache(CU.ATOMICS_CACHE_NAME);
-            }
+            IgniteInternalCache atomicsCache = ctx.cache().atomicsCache();
 
             assert atomicsCache != null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6d110437/modules/core/src/test/java/org/apache/ignite/cache/affinity/IgniteClientNodeAffinityTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/IgniteClientNodeAffinityTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/IgniteClientNodeAffinityTest.java
index f0af2c1..467349f 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/IgniteClientNodeAffinityTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/IgniteClientNodeAffinityTest.java
@@ -55,9 +55,6 @@ public class IgniteClientNodeAffinityTest extends GridCommonAbstractTest {
     /** */
     private static final String CACHE4 = "cache4";
 
-    /** */
-    private static final String CACHE5 = "cache5";
-
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6d110437/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
new file mode 100644
index 0000000..9745ad8
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
@@ -0,0 +1,213 @@
+/*
+ * 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.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.managers.discovery.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.*;
+import org.apache.ignite.internal.processors.cache.distributed.near.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.concurrent.*;
+
+/**
+ * Tests that cache specified in configuration start on client nodes.
+ */
+public class IgniteDynamicClientCacheStartSelfTest extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private CacheConfiguration ccfg;
+
+    /** */
+    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);
+
+        if (ccfg != null)
+            cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConfiguredCacheOnClientNode() throws Exception {
+        ccfg = new CacheConfiguration();
+
+        final String cacheName = null;
+
+        Ignite ignite0 = startGrid(0);
+
+        checkCache(ignite0, cacheName, true, false);
+
+        client = true;
+
+        Ignite ignite1 = startGrid(1);
+
+        checkCache(ignite1, cacheName, false, false);
+
+        ccfg = new CacheConfiguration();
+
+        ccfg.setNearConfiguration(new NearCacheConfiguration());
+
+        Ignite ignite2 = startGrid(2);
+
+        checkCache(ignite2, cacheName, false, true);
+
+        ccfg = null;
+
+        Ignite ignite3 = startGrid(3);
+
+        checkNoCache(ignite3, cacheName);
+
+        assertNotNull(ignite3.cache(cacheName));
+
+        checkCache(ignite3, cacheName, false, false);
+
+        Ignite ignite4 = startGrid(4);
+
+        checkNoCache(ignite4, cacheName);
+
+        assertNotNull(ignite4.createNearCache(cacheName, new NearCacheConfiguration<>()));
+
+        checkCache(ignite4, cacheName, false, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNearCacheStartError() throws Exception {
+        ccfg = new CacheConfiguration();
+
+        final String cacheName = null;
+
+        Ignite ignite0 = startGrid(0);
+
+        checkCache(ignite0, cacheName, true, false);
+
+        client = true;
+
+        final Ignite ignite1 = startGrid(1);
+
+        checkCache(ignite1, cacheName, false, false);
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                ignite1.getOrCreateNearCache(cacheName, new NearCacheConfiguration<>());
+
+                return null;
+            }
+        }, IgniteException.class, null);
+
+        checkCache(ignite1, cacheName, false, false);
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                ignite1.createNearCache(cacheName, new NearCacheConfiguration<>());
+
+                return null;
+            }
+        }, IgniteException.class, null);
+
+        checkCache(ignite1, cacheName, false, false);
+    }
+
+    /**
+     * @param ignite Node.
+     * @param cacheName Cache name
+     * @param srv {@code True} if server cache is expected.
+     * @param near {@code True} if near cache is expected.
+     */
+    private void checkCache(Ignite ignite, String cacheName, boolean srv, boolean near) {
+        GridCacheAdapter<Object, Object> cache = ((IgniteKernal)ignite).context().cache().internalCache(cacheName);
+
+        assertNotNull("No cache on node " + ignite.name(), cache);
+
+        assertEquals(near, cache.context().isNear());
+
+        if (near)
+            cache = ((GridNearCacheAdapter)cache).dht();
+
+        if (srv)
+            assertSame(GridCacheConcurrentMap.class, cache.map().getClass());
+        else
+            assertSame(GridNoStorageCacheMap.class, cache.map().getClass());
+
+        ClusterNode node = ((IgniteKernal)ignite).localNode();
+
+        for (Ignite ignite0 : Ignition.allGrids()) {
+            GridDiscoveryManager disco = ((IgniteKernal)ignite0).context().discovery();
+
+            assertTrue(disco.cacheNode(node, cacheName));
+            assertEquals(srv, disco.cacheAffinityNode(node, cacheName));
+            assertEquals(near, disco.cacheNearNode(node, cacheName));
+
+            if (srv)
+                assertTrue(ignite0.affinity(null).primaryPartitions(node).length > 0);
+            else
+                assertEquals(0, ignite0.affinity(null).primaryPartitions(node).length);
+        }
+
+        assertNotNull(ignite.cache(cacheName));
+    }
+
+    /**
+     * @param ignite Node.
+     * @param cacheName Cache name.
+     */
+    private void checkNoCache(Ignite ignite, String cacheName) {
+        GridCacheAdapter<Object, Object> cache = ((IgniteKernal)ignite).context().cache().internalCache(cacheName);
+
+        assertNull("Unexpected cache on node " + ignite.name(), cache);
+
+        ClusterNode node = ((IgniteKernal)ignite).localNode();
+
+        for (Ignite ignite0 : Ignition.allGrids()) {
+            GridDiscoveryManager disco = ((IgniteKernal)ignite0).context().discovery();
+
+            assertFalse(disco.cacheNode(node, cacheName));
+            assertFalse(disco.cacheAffinityNode(node, cacheName));
+            assertFalse(disco.cacheNearNode(node, cacheName));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6d110437/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteSystemCacheOnClientTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteSystemCacheOnClientTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteSystemCacheOnClientTest.java
index 52ef7e2..a7b2df6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteSystemCacheOnClientTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteSystemCacheOnClientTest.java
@@ -76,7 +76,7 @@ public class IgniteSystemCacheOnClientTest extends GridCommonAbstractTest {
 
         assertNotNull(marshCache);
 
-        assertTrue("Marshaller cache on client should have near cache", marshCache.context().isNear());
+        assertFalse(marshCache.context().isNear());
 
         marshCache = ((IgniteKernal)ignite(0)).internalCache(CU.MARSH_CACHE_NAME);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6d110437/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index b3eb899..18cc453 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -101,6 +101,7 @@ public class IgniteCacheTestSuite4 extends TestSuite {
         suite.addTestSuite(IgniteCacheDynamicStopSelfTest.class);
         suite.addTestSuite(IgniteCacheConfigurationTemplateTest.class);
         suite.addTestSuite(IgniteCacheConfigurationDefaultTemplateTest.class);
+        suite.addTestSuite(IgniteDynamicClientCacheStartSelfTest.class);
 
         suite.addTestSuite(GridCacheTxLoadFromStoreOnLockSelfTest.class);
 



[03/29] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-709_2' into ignite-709_3

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-709_2' into ignite-709_3


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

Branch: refs/heads/ignite-709_2
Commit: 1ede1d0ad9dfb707d3986d34aa4e2b188ece7129
Parents: 22ec4aa cb762a4
Author: sboikov <sb...@gridgain.com>
Authored: Thu May 14 14:42:57 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu May 14 14:42:57 2015 +0300

----------------------------------------------------------------------
 .../dht/preloader/GridDhtPartitionsExchangeFuture.java          | 2 +-
 .../org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java    | 3 +++
 .../apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java | 5 -----
 3 files changed, 4 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1ede1d0a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------


[04/29] incubator-ignite git commit: # ignite-709_3 do not create system caches on client nodes

Posted by sb...@apache.org.
# ignite-709_3 do not create system caches on client nodes


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

Branch: refs/heads/ignite-709_2
Commit: 7ed0ddf29fc9baae66ca2fb9e3be960e0b7e295b
Parents: 1ede1d0
Author: sboikov <sb...@gridgain.com>
Authored: Thu May 14 14:58:39 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu May 14 14:58:39 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    | 21 ++++-----
 .../datastructures/DataStructuresProcessor.java | 10 ++++-
 .../IgniteClientDataStructuresAbstractTest.java | 47 ++++++++++++++++++++
 3 files changed, 67 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7ed0ddf2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index e63dd8b..8771515 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -736,13 +736,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (marshallerCache() == null) {
             assert ctx.config().isClientMode() : "Marshaller cache is missed on server node.";
 
-            DynamicCacheDescriptor desc = registeredCaches.get(CU.MARSH_CACHE_NAME);
-
-            assert desc != null && desc.cacheConfiguration() != null && desc.cacheType().equals(CacheType.MARSHALLER);
-
-            // On client node user near-only marshaller cache.
-            IgniteInternalFuture<?> fut = dynamicStartCache(desc.cacheConfiguration(),
-                CU.MARSH_CACHE_NAME, new NearCacheConfiguration(), desc.cacheType(), false);
+            // On client node use near-only marshaller cache.
+            IgniteInternalFuture<?> fut = startCacheAsync(CU.MARSH_CACHE_NAME, new NearCacheConfiguration(), false);
 
             assert fut != null;
 
@@ -2554,7 +2549,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (internalCache(CU.UTILITY_CACHE_NAME) != null)
             return new GridFinishedFuture<>();
 
-        return startCacheAsync(CU.UTILITY_CACHE_NAME, true);
+        return startCacheAsync(CU.UTILITY_CACHE_NAME, null, true);
     }
 
     /**
@@ -2634,7 +2629,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If failed.
      */
     private IgniteCache startJCache(String cacheName, boolean failIfNotStarted) throws IgniteCheckedException {
-        IgniteInternalFuture<?> fut = startCacheAsync(cacheName, failIfNotStarted);
+        IgniteInternalFuture<?> fut = startCacheAsync(cacheName, null, failIfNotStarted);
 
         if (fut != null) {
             fut.get();
@@ -2654,11 +2649,15 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /**
      * @param cacheName Cache name.
+     * @param nearCfg Near cache configuration.
      * @param failIfNotStarted If {@code true} throws {@link IllegalArgumentException} if cache is not started,
      *        otherwise returns {@code null} in this case.
      * @return Future.
      */
-    @Nullable private IgniteInternalFuture<?> startCacheAsync(String cacheName, boolean failIfNotStarted) {
+    @Nullable public IgniteInternalFuture<?> startCacheAsync(String cacheName,
+        @Nullable NearCacheConfiguration nearCfg,
+        boolean failIfNotStarted)
+    {
         String masked = maskNull(cacheName);
 
         DynamicCacheDescriptor desc = registeredCaches.get(masked);
@@ -2686,6 +2685,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         req.clientStartOnly(true);
 
+        req.nearCacheConfiguration(nearCfg);
+
         return F.first(initiateCacheChanges(F.asList(req)));
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7ed0ddf2/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index b8d5bef..0a90e32 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -123,7 +123,15 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
         assert utilityCache != null;
 
         if (atomicCfg != null) {
-            IgniteInternalCache atomicsCache = ctx.cache().getOrStartCache(CU.ATOMICS_CACHE_NAME);
+            IgniteInternalCache atomicsCache = ctx.cache().internalCache(CU.ATOMICS_CACHE_NAME);
+
+            if (atomicsCache == null) {
+                assert ctx.config().isClientMode() : "Atomics cache is missed on server node.";
+
+                ctx.cache().startCacheAsync(CU.ATOMICS_CACHE_NAME, new NearCacheConfiguration(), true).get();
+
+                atomicsCache = ctx.cache().internalCache(CU.ATOMICS_CACHE_NAME);
+            }
 
             assert atomicsCache != null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7ed0ddf2/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
index 175c1e6..db16062 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
@@ -19,13 +19,17 @@ package org.apache.ignite.internal.processors.cache.datastructures;
 
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.junits.common.*;
 
 import java.util.*;
+import java.util.concurrent.*;
 
 /**
  *
@@ -183,6 +187,49 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testLatch() throws Exception {
+        Ignite clientNode = clientIgnite();
+
+        final Ignite srvNode = serverNode();
+
+        assertNull(clientNode.countDownLatch("latch1", 1, true, false));
+
+        try (IgniteCountDownLatch latch = clientNode.countDownLatch("latch1", 1, true, true)) {
+            assertNotNull(latch);
+
+            assertEquals(1, latch.count());
+
+            IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    U.sleep(1000);
+
+                    IgniteCountDownLatch latch0 = srvNode.countDownLatch("latch1", 1, true, false);
+
+                    assertEquals(1, latch0.count());
+
+                    log.info("Count down latch.");
+
+                    latch0.countDown();
+
+                    assertEquals(0, latch0.count());
+
+                    return null;
+                }
+            });
+
+            log.info("Await latch.");
+
+            assertTrue(latch.await(5000, TimeUnit.MILLISECONDS));
+
+            log.info("Finished wait.");
+
+            fut.get();
+        }
+    }
+
+    /**
      * @return Client node.
      */
     private Ignite clientIgnite() {


[29/29] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-709_2' into ignite-709_2

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-709_2' into ignite-709_2


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

Branch: refs/heads/ignite-709_2
Commit: c1fd8e7060b7a395eb9369be95888f7db134b558
Parents: 69fcdf5 ed76bf0
Author: sboikov <sb...@gridgain.com>
Authored: Mon May 18 12:30:55 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon May 18 12:30:55 2015 +0300

----------------------------------------------------------------------
 .../internal/interop/InteropIgnition.java       | 65 +++++++++++++++++++-
 .../internal/interop/InteropProcessor.java      | 13 +++-
 .../discovery/tcp/TcpClientDiscoverySpi.java    | 22 +++++--
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 39 +++++-------
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   | 22 ++++---
 .../messages/TcpDiscoveryAbstractMessage.java   | 17 -----
 6 files changed, 122 insertions(+), 56 deletions(-)
----------------------------------------------------------------------



[18/29] incubator-ignite git commit: # ignite-709_3

Posted by sb...@apache.org.
# ignite-709_3


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

Branch: refs/heads/ignite-709_2
Commit: 24bb2959a67201e5be9d5d01467a1d5731daf886
Parents: 33459b1
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 15 17:45:53 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 15 17:47:02 2015 +0300

----------------------------------------------------------------------
 .../internal/GridProjectionForCachesSelfTest.java      | 11 ++++++++---
 .../discovery/GridDiscoveryManagerSelfTest.java        | 13 +++++++------
 2 files changed, 15 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/24bb2959/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionForCachesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionForCachesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionForCachesSelfTest.java
index 0a961d4..04cd3f0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionForCachesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionForCachesSelfTest.java
@@ -52,9 +52,14 @@ public class GridProjectionForCachesSelfTest extends GridCommonAbstractTest {
 
         cfg.setDiscoverySpi(discoverySpi());
 
-        cfg.setCacheConfiguration(
-            cacheConfiguration(null, new AttributeFilter(getTestGridName(0)), false),
-            cacheConfiguration(CACHE_NAME, new AttributeFilter(getTestGridName(2), getTestGridName(3)), true));
+        List<CacheConfiguration> ccfgs = new ArrayList<>();
+
+        if (gridName.equals(getTestGridName(0)))
+            ccfgs.add(cacheConfiguration(null, new AttributeFilter(getTestGridName(0)), false));
+        else if (gridName.equals(getTestGridName(2)) || gridName.equals(getTestGridName(3)))
+            ccfgs.add(cacheConfiguration(CACHE_NAME, new AttributeFilter(getTestGridName(2), getTestGridName(3)), true));
+
+        cfg.setCacheConfiguration(ccfgs.toArray(new CacheConfiguration[ccfgs.size()]));
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/24bb2959/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
index 6f7c935..be5b9c6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
@@ -66,14 +66,15 @@ public class GridDiscoveryManagerSelfTest extends GridCommonAbstractTest {
 
         if (gridName.equals(getTestGridName(1)))
             cfg.setClientMode(true);
+        else {
+            ccfg1.setNearConfiguration(null);
+            ccfg2.setNearConfiguration(null);
 
-        ccfg1.setNearConfiguration(null);
-        ccfg2.setNearConfiguration(null);
+            ccfg1.setCacheMode(PARTITIONED);
+            ccfg2.setCacheMode(PARTITIONED);
 
-        ccfg1.setCacheMode(PARTITIONED);
-        ccfg2.setCacheMode(PARTITIONED);
-
-        cfg.setCacheConfiguration(ccfg1, ccfg2);
+            cfg.setCacheConfiguration(ccfg1, ccfg2);
+        }
 
         return cfg;
     }


[25/29] incubator-ignite git commit: # ignite-709_3

Posted by sb...@apache.org.
# ignite-709_3


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

Branch: refs/heads/ignite-709_2
Commit: 1d8ae1612471731136e7b5c19d6cdfbd6b9b61bc
Parents: 46b36e8
Author: sboikov <se...@inria.fr>
Authored: Fri May 15 20:52:51 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Fri May 15 20:52:51 2015 +0300

----------------------------------------------------------------------
 .../ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1d8ae161/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
index 2a5616f..7fb4563 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
@@ -113,7 +113,7 @@ public class IgniteCacheDataStructuresSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(IgniteDataStructureUniqueNameTest.class));
 
         suite.addTest(new TestSuite(IgniteClientDataStructuresTest.class));
-        // suite.addTest(new TestSuite(IgniteClientDiscoveryDataStructuresTest.class));
+        suite.addTest(new TestSuite(IgniteClientDiscoveryDataStructuresTest.class));
 
         return suite;
     }


[10/29] incubator-ignite git commit: # ignite-709_3

Posted by sb...@apache.org.
# ignite-709_3


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

Branch: refs/heads/ignite-709_2
Commit: 64ed3f19c1a3c65b4d47f806a212c52bfabb3dae
Parents: fb1d79c
Author: sboikov <se...@inria.fr>
Authored: Thu May 14 21:55:51 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Thu May 14 21:55:51 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    | 144 ++++++++++++-------
 .../preloader/GridDhtPartitionDemandPool.java   |   2 +-
 .../datastructures/DataStructuresProcessor.java |   2 +-
 3 files changed, 91 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/64ed3f19/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 9319b45..331e454 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -127,6 +127,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** Count down latch for caches. */
     private final CountDownLatch cacheStartedLatch = new CountDownLatch(1);
 
+    /** */
+    private final GridFutureAdapter<Object> sysCacheStartFut = new GridFutureAdapter<>();
+
     /**
      * @param ctx Kernal context.
      */
@@ -661,6 +664,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void onKernalStart() throws IgniteCheckedException {
+        DynamicCacheDescriptor marshCacheDesc = null;
+        DynamicCacheDescriptor utilityCacheDesc = null;
+        DynamicCacheDescriptor atomicsCacheDesc = null;
+
         try {
             if (ctx.config().isDaemon())
                 return;
@@ -727,17 +734,34 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                     jCacheProxies.put(maskNull(name), new IgniteCacheProxy(ctx, cache, null, false));
                 }
+
+                if (CU.MARSH_CACHE_NAME.equals(ccfg.getName()))
+                    marshCacheDesc = desc;
+                else if (CU.UTILITY_CACHE_NAME.equals(ccfg.getName()))
+                    utilityCacheDesc = desc;
+                else if (CU.ATOMICS_CACHE_NAME.equals(ccfg.getName()))
+                    atomicsCacheDesc = desc;
             }
         }
         finally {
             cacheStartedLatch.countDown();
         }
 
-        if (marshallerCache() == null) {
-            IgniteInternalFuture<?> fut = marshallerCacheAsync();
+        if (ctx.config().isClientMode()) {
+            assert marshCacheDesc != null;
+            assert utilityCacheDesc != null;
+            assert atomicsCacheDesc != null;
+
+            Collection<DynamicCacheChangeRequest> reqs = new ArrayList<>();
 
-            fut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> fut) {
+            reqs.add(clientSystemCacheRequest(marshCacheDesc, new NearCacheConfiguration()));
+            reqs.add(clientSystemCacheRequest(utilityCacheDesc, null));
+            reqs.add(clientSystemCacheRequest(atomicsCacheDesc, new NearCacheConfiguration()));
+
+            startClientSystemCaches(reqs);
+
+            sysCacheStartFut.listen(new CI1<IgniteInternalFuture<Object>>() {
+                @Override public void apply(IgniteInternalFuture<Object> fut) {
                     try {
                         marshallerCacheCallbacks();
                     }
@@ -747,8 +771,11 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 }
             });
         }
-        else
+        else {
+            sysCacheStartFut.onDone();
+
             marshallerCacheCallbacks();
+        }
 
         // Must call onKernalStart on shared managers after creation of fetched caches.
         for (GridCacheSharedManager<?, ?> mgr : sharedCtx.managers())
@@ -800,6 +827,51 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param reqs Start requests.
+     */
+    private void startClientSystemCaches(Collection<DynamicCacheChangeRequest> reqs) {
+        assert !F.isEmpty(reqs) : reqs;
+
+        GridCompoundFuture<Object, Object> fut = new GridCompoundFuture<>();
+
+        for (DynamicCacheStartFuture startFut : initiateCacheChanges(reqs))
+            fut.add(startFut);
+
+        fut.markInitialized();
+
+        fut.listen(new CI1<IgniteInternalFuture<Object>>() {
+            @Override public void apply(IgniteInternalFuture<Object> fut) {
+                sysCacheStartFut.onDone();
+            }
+        });
+    }
+
+    /**
+     * @param cacheDesc Cache descriptor.
+     * @return Cache change request.
+     */
+    private DynamicCacheChangeRequest clientSystemCacheRequest(
+        DynamicCacheDescriptor cacheDesc,
+        @Nullable NearCacheConfiguration nearCfg)
+    {
+        DynamicCacheChangeRequest desc = new DynamicCacheChangeRequest(
+            cacheDesc.cacheConfiguration().getName(),
+            ctx.localNodeId());
+
+        desc.clientStartOnly(true);
+
+        desc.nearCacheConfiguration(nearCfg);
+
+        desc.deploymentId(cacheDesc.deploymentId());
+
+        desc.startCacheConfiguration(cacheDesc.cacheConfiguration());
+
+        desc.cacheType(cacheDesc.cacheType());
+
+        return desc;
+    }
+
+    /**
      * @throws IgniteCheckedException If failed.
      */
     private void marshallerCacheCallbacks() throws IgniteCheckedException {
@@ -856,6 +928,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     @Override public void onKernalStop(boolean cancel) {
         cacheStartedLatch.countDown();
 
+        sysCacheStartFut.onDone();
+
         if (ctx.config().isDaemon())
             return;
 
@@ -2534,7 +2608,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         assert ctx.config().isClientMode() : "Utility cache is missed on server node.";
 
-        getOrStartCache(CU.UTILITY_CACHE_NAME);
+        sysCacheStartFut.get();
 
         return internalCache(CU.UTILITY_CACHE_NAME);
     }
@@ -2542,24 +2616,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /**
      * @return Utility cache start future.
      */
-    public IgniteInternalFuture<?> utilityCacheAsync() {
-        if (internalCache(CU.UTILITY_CACHE_NAME) != null)
-            return new GridFinishedFuture<>();
-
-        return startCacheAsync(CU.UTILITY_CACHE_NAME, null, true);
-    }
-
-    /**
-     * @return Utility cache start future.
-     */
-    public IgniteInternalFuture<?> marshallerCacheAsync() {
-        if (internalCache(CU.MARSH_CACHE_NAME) != null)
-            return new GridFinishedFuture<>();
-
-        assert ctx.config().isClientMode() : "Marshaller cache is missed on server node.";
-
-        // On client node use near-only marshaller cache.
-        return startCacheAsync(CU.MARSH_CACHE_NAME, new NearCacheConfiguration(), false);
+    public IgniteInternalFuture<?> systemCachesStartFuture() {
+        return sysCacheStartFut;
     }
 
     /**
@@ -2639,35 +2697,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If failed.
      */
     private IgniteCache startJCache(String cacheName, boolean failIfNotStarted) throws IgniteCheckedException {
-        IgniteInternalFuture<?> fut = startCacheAsync(cacheName, null, failIfNotStarted);
-
-        if (fut != null) {
-            fut.get();
-
-            String masked = maskNull(cacheName);
-
-            IgniteCache cache = jCacheProxies.get(masked);
-
-            if (cache == null && failIfNotStarted)
-                throw new IllegalArgumentException("Cache is not started: " + cacheName);
-
-            return cache;
-        }
-
-        return null;
-    }
-
-    /**
-     * @param cacheName Cache name.
-     * @param nearCfg Near cache configuration.
-     * @param failIfNotStarted If {@code true} throws {@link IllegalArgumentException} if cache is not started,
-     *        otherwise returns {@code null} in this case.
-     * @return Future.
-     */
-    @Nullable public IgniteInternalFuture<?> startCacheAsync(String cacheName,
-        @Nullable NearCacheConfiguration nearCfg,
-        boolean failIfNotStarted)
-    {
         String masked = maskNull(cacheName);
 
         DynamicCacheDescriptor desc = registeredCaches.get(masked);
@@ -2695,9 +2724,14 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         req.clientStartOnly(true);
 
-        req.nearCacheConfiguration(nearCfg);
+        F.first(initiateCacheChanges(F.asList(req))).get();
 
-        return F.first(initiateCacheChanges(F.asList(req)));
+        IgniteCache cache = jCacheProxies.get(masked);
+
+        if (cache == null && failIfNotStarted)
+            throw new IllegalArgumentException("Cache is not started: " + cacheName);
+
+        return cache;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/64ed3f19/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
index ba2af9e..4153c5f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
@@ -824,7 +824,7 @@ public class GridDhtPartitionDemandPool<K, V> {
                         log.debug("Waiting for marshaller cache preload [cacheName=" + cctx.name() + ']');
 
                     try {
-                        cctx.kernalContext().cache().marshallerCacheAsync().get();
+                        cctx.kernalContext().cache().systemCachesStartFuture().get();
 
                         cctx.kernalContext().cache().marshallerCache().context().awaitStarted();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/64ed3f19/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 0a90e32..b6d4b40 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -128,7 +128,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
             if (atomicsCache == null) {
                 assert ctx.config().isClientMode() : "Atomics cache is missed on server node.";
 
-                ctx.cache().startCacheAsync(CU.ATOMICS_CACHE_NAME, new NearCacheConfiguration(), true).get();
+                ctx.cache().systemCachesStartFuture().get();
 
                 atomicsCache = ctx.cache().internalCache(CU.ATOMICS_CACHE_NAME);
             }


[20/29] incubator-ignite git commit: # ignite-709_3

Posted by sb...@apache.org.
# ignite-709_3


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

Branch: refs/heads/ignite-709_2
Commit: 994cb14cf0a3ad76e4294d52469cfa4f473a8df2
Parents: 77ef2e9
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 15 17:56:32 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 15 17:56:32 2015 +0300

----------------------------------------------------------------------
 .../ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/994cb14c/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
index 7fb4563..2a5616f 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
@@ -113,7 +113,7 @@ public class IgniteCacheDataStructuresSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(IgniteDataStructureUniqueNameTest.class));
 
         suite.addTest(new TestSuite(IgniteClientDataStructuresTest.class));
-        suite.addTest(new TestSuite(IgniteClientDiscoveryDataStructuresTest.class));
+        // suite.addTest(new TestSuite(IgniteClientDiscoveryDataStructuresTest.class));
 
         return suite;
     }


[28/29] incubator-ignite git commit: # ignite-709_3

Posted by sb...@apache.org.
# ignite-709_3


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

Branch: refs/heads/ignite-709_2
Commit: 69fcdf55c008a85ee4ad1f3828a04bd0fc6c419b
Parents: dc315c0
Author: sboikov <sb...@gridgain.com>
Authored: Mon May 18 10:20:50 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon May 18 10:20:50 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheVariableTopologySelfTest.java        | 12 ++++++++++--
 1 file changed, 10 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/69fcdf55/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java
index ce0a55c..dc50ee6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java
@@ -19,8 +19,10 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -137,8 +139,14 @@ public class GridCacheVariableTopologySelfTest extends GridCommonAbstractTest {
 
                         tx.commit();
                     }
-                    catch (TransactionOptimisticException e) {
-                        info("Caught cache optimistic exception: " + e);
+                    catch (ClusterTopologyException e) {
+                        info("Caught topology exception: " + e);
+                    }
+                    catch (IgniteException e) {
+                        if (X.hasCause(e, ClusterTopologyCheckedException.class))
+                            info("Caught cache exception: " + e);
+                        else
+                            throw e;
                     }
 
                     try {


[19/29] incubator-ignite git commit: # ignite-709_3

Posted by sb...@apache.org.
# ignite-709_3


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

Branch: refs/heads/ignite-709_2
Commit: 77ef2e99f827c5434728134603a0159c7bb66fce
Parents: 24bb295
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 15 17:49:40 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 15 17:49:40 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/distributed/near/GridNearGetFuture.java       | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/77ef2e99/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index fc178e3..74438bb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -274,7 +274,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
         if (affNodes.isEmpty()) {
             assert !cctx.affinityNode();
 
-            onDone(new ClusterTopologyCheckedException("Failed to map keys for near-only cache (all partition " +
+            onDone(new ClusterTopologyServerNotFoundException("Failed to map keys for near-only cache (all partition " +
                 "nodes left the grid)."));
 
             return;