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

[01/50] incubator-ignite git commit: #ignite-237: remove consistent hash from IgniteCacheAffinitySelfTest.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-45 38ebb3e3a -> 647691f17


#ignite-237: remove consistent hash from IgniteCacheAffinitySelfTest.


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

Branch: refs/heads/ignite-45
Commit: df3c76e4a353c599832a105ce0fc22f73104edf1
Parents: fecb680
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Feb 26 15:53:43 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Feb 26 15:53:43 2015 +0300

----------------------------------------------------------------------
 .../ignite/IgniteCacheAffinitySelfTest.java     | 30 +++++++-------------
 1 file changed, 11 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/df3c76e4/modules/core/src/test/java/org/apache/ignite/IgniteCacheAffinitySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/IgniteCacheAffinitySelfTest.java b/modules/core/src/test/java/org/apache/ignite/IgniteCacheAffinitySelfTest.java
index 7186ae1..c3e76a6 100644
--- a/modules/core/src/test/java/org/apache/ignite/IgniteCacheAffinitySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/IgniteCacheAffinitySelfTest.java
@@ -19,7 +19,6 @@ package org.apache.ignite;
 
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cache.affinity.fair.*;
 import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
@@ -40,13 +39,10 @@ public class IgniteCacheAffinitySelfTest extends IgniteCacheAbstractTest {
     private int GRID_COUNT = 3;
 
     /** Cache name */
-    private final String CACHE1 = "ConsistentHash";
+    private final String CACHE1 = "Fair";
 
     /** Cache name */
-    private final String CACHE2 = "Fair";
-
-    /** Cache name */
-    private final String CACHE3 = "Rendezvous";
+    private final String CACHE2 = "Rendezvous";
 
     /** {@inheritDoc} */
     @Override protected int gridCount() {
@@ -61,20 +57,16 @@ public class IgniteCacheAffinitySelfTest extends IgniteCacheAbstractTest {
 
         CacheConfiguration cache1 = cacheConfiguration(null);
         cache1.setName(CACHE1);
-        cache1.setAffinity(new CacheConsistentHashAffinityFunction());
+        cache1.setAffinity(new CachePartitionFairAffinity());
 
         CacheConfiguration cache2 = cacheConfiguration(null);
         cache2.setName(CACHE2);
-        cache2.setAffinity(new CachePartitionFairAffinity());
-
-        CacheConfiguration cache3 = cacheConfiguration(null);
-        cache3.setName(CACHE3);
-        cache3.setAffinity(new CacheRendezvousAffinityFunction());
+        cache2.setAffinity(new CacheRendezvousAffinityFunction());
 
         if (gridName.contains("0"))
             cfg.setCacheConfiguration(cache0);
         else
-            cfg.setCacheConfiguration(cache0, cache1, cache2, cache3);
+            cfg.setCacheConfiguration(cache0, cache1, cache2);
 
         return cfg;
     }
@@ -116,8 +108,8 @@ public class IgniteCacheAffinitySelfTest extends IgniteCacheAbstractTest {
     private void checkAffinity() {
         checkAffinity(grid(0).affinity(null), cache(1, null).affinity());
         checkAffinity(grid(0).affinity(CACHE1), cache(1, CACHE1).affinity());
+        checkAffinity(grid(0).affinity(CACHE1), cache(1, CACHE1).affinity());
         checkAffinity(grid(0).affinity(CACHE2), cache(1, CACHE2).affinity());
-        checkAffinity(grid(0).affinity(CACHE3), cache(1, CACHE3).affinity());
     }
 
     /**
@@ -226,13 +218,13 @@ public class IgniteCacheAffinitySelfTest extends IgniteCacheAbstractTest {
 
         Collection<Integer> col1 = new HashSet<>();
 
-        for (int i = 0; i < arr1.length; ++i)
-            col1.add(arr1[i]);
+        for (int anArr1 : arr1)
+            col1.add(anArr1);
 
-        for (int i = 0; i < arr2.length; ++i) {
-            assertTrue(col1.contains(arr2[i]));
+        for (int anArr2 : arr2) {
+            assertTrue(col1.contains(anArr2));
 
-            col1.remove(arr2[i]);
+            col1.remove(anArr2);
         }
 
         assertEquals(0, col1.size());


[46/50] incubator-ignite git commit: IGNITE-45 - Reshuffling code to reuse in dynamic cache start.

Posted by ag...@apache.org.
IGNITE-45 - Reshuffling code to reuse in dynamic cache start.


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

Branch: refs/heads/ignite-45
Commit: 501bd5c39d7626094dad2e4068f823a4e05dcf42
Parents: fe2985b
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Tue Mar 3 15:20:48 2015 -0800
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Tue Mar 3 15:20:48 2015 -0800

----------------------------------------------------------------------
 .../affinity/AffinityTopologyVersion.java       |  12 +-
 .../processors/cache/GridCacheProcessor.java    | 964 ++++++++++---------
 .../cache/GridCacheSharedContext.java           |  14 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   4 +-
 4 files changed, 522 insertions(+), 472 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/501bd5c3/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
index fc5f193..be6fae5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
@@ -35,6 +35,13 @@ public class AffinityTopologyVersion implements Comparable<AffinityTopologyVersi
     private long topVer;
 
     /**
+     * Empty constructor required by {@link Externalizable}.
+     */
+    public AffinityTopologyVersion() {
+        // No-op.
+    }
+
+    /**
      * @param ver Version.
      */
     public AffinityTopologyVersion(long ver) {
@@ -69,10 +76,7 @@ public class AffinityTopologyVersion implements Comparable<AffinityTopologyVersi
 
     /** {@inheritDoc} */
     @Override public boolean equals(Object o) {
-        if (!(o instanceof AffinityTopologyVersion))
-            return false;
-
-        return topVer == ((AffinityTopologyVersion)o).topVer;
+        return o instanceof AffinityTopologyVersion && topVer == ((AffinityTopologyVersion)o).topVer;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/501bd5c3/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 5b4852e..260cab0 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
@@ -111,6 +111,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** Dynamic caches. */
     private ConcurrentMap<String, DynamicCacheDescriptor> dynamicCaches = new ConcurrentHashMap<>();
 
+    /** */
+    private IdentityHashMap<CacheStore, ThreadLocal> sesHolders = new IdentityHashMap<>();
+
     /**
      * @param ctx Kernal context.
      */
@@ -592,358 +595,598 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         Collection<GridCacheAdapter<?, ?>> startSeq = new ArrayList<>(cfgs.length);
 
-        IdentityHashMap<CacheStore, ThreadLocal> sesHolders = new IdentityHashMap<>();
-
         for (int i = 0; i < cfgs.length; i++) {
             CacheConfiguration<?, ?> cfg = new CacheConfiguration(cfgs[i]);
 
             // Initialize defaults.
             initialize(cfg);
 
-            CacheStore cfgStore = cfg.getCacheStoreFactory() != null ? cfg.getCacheStoreFactory().create() : null;
+            cfgs[i] = cfg; // Replace original configuration value.
 
-            validate(ctx.config(), cfg, cfgStore);
+            if (caches.containsKey(cfg.getName())) {
+                String cacheName = cfg.getName();
 
-            CacheJtaManagerAdapter jta = JTA.create(cfg.getTransactionManagerLookupClassName() == null);
+                if (cacheName != null)
+                    throw new IgniteCheckedException("Duplicate cache name found (check configuration and " +
+                        "assign unique name to each cache): " + cacheName);
+                else
+                    throw new IgniteCheckedException("Default cache has already been configured (check configuration and " +
+                        "assign unique name to each cache).");
+            }
 
-            jta.createTmLookup(cfg);
+            GridCacheContext cacheCtx = createCache(cfg);
 
-            // Skip suggestions for system caches.
-            if (!sysCaches.contains(cfg.getName()))
-                suggestOptimizations(cfg, cfgStore != null);
+            sharedCtx.addCacheContext(cacheCtx);
 
-            List<Object> toPrepare = new ArrayList<>();
+            startSeq.add(cacheCtx.cache());
 
-            toPrepare.add(jta.tmLookup());
-            toPrepare.add(cfgStore);
+            caches.put(cfg.getName(), cacheCtx.cache());
 
-            if (cfgStore instanceof GridCacheLoaderWriterStore) {
-                toPrepare.add(((GridCacheLoaderWriterStore)cfgStore).loader());
-                toPrepare.add(((GridCacheLoaderWriterStore)cfgStore).writer());
-            }
+            if (sysCaches.contains(cfg.getName()))
+                stopSeq.addLast(cacheCtx.cache());
+            else
+                stopSeq.addFirst(cacheCtx.cache());
+        }
 
-            prepare(cfg, toPrepare.toArray(new Object[toPrepare.size()]));
+        // Start shared managers.
+        for (GridCacheSharedManager mgr : sharedCtx.managers())
+            mgr.start(sharedCtx);
 
-            U.startLifecycleAware(lifecycleAwares(cfg, jta.tmLookup(), cfgStore));
+        for (GridCacheAdapter<?, ?> cache : startSeq)
+            startCache(cache);
 
-            // Init default key mapper.
-            CacheAffinityKeyMapper dfltAffMapper;
+        for (Map.Entry<String, GridCacheAdapter<?, ?>> e : caches.entrySet()) {
+            GridCacheAdapter cache = e.getValue();
 
-            if (cfg.getAffinityMapper().getClass().equals(GridCacheDefaultAffinityKeyMapper.class))
-                dfltAffMapper = cfg.getAffinityMapper();
-            else {
-                dfltAffMapper = new GridCacheDefaultAffinityKeyMapper();
+            proxies.put(e.getKey(), new GridCacheProxyImpl(cache.context(), cache, null));
 
-                prepare(cfg, dfltAffMapper, false);
-            }
+            jCacheProxies.put(e.getKey(), new IgniteCacheProxy(cache.context(), cache, null, false));
+        }
 
-            cfgs[i] = cfg; // Replace original configuration value.
+        // Internal caches which should not be returned to user.
+        for (Map.Entry<String, GridCacheAdapter<?, ?>> e : caches.entrySet()) {
+            GridCacheAdapter cache = e.getValue();
 
-            GridCacheAffinityManager affMgr = new GridCacheAffinityManager();
-            GridCacheEventManager evtMgr = new GridCacheEventManager();
-            GridCacheSwapManager swapMgr = new GridCacheSwapManager(cfg.getCacheMode() == LOCAL || !GridCacheUtils.isNearEnabled(cfg));
-            GridCacheEvictionManager evictMgr = new GridCacheEvictionManager();
-            GridCacheQueryManager qryMgr = queryManager(cfg);
-            CacheContinuousQueryManager contQryMgr = new CacheContinuousQueryManager();
-            CacheDataStructuresManager dataStructuresMgr = new CacheDataStructuresManager();
-            GridCacheTtlManager ttlMgr = new GridCacheTtlManager();
-            GridCacheDrManager drMgr = ctx.createComponent(GridCacheDrManager.class);
-            IgniteCacheSerializationManager serMgr = ctx.createComponent(IgniteCacheSerializationManager.class);
+            if (!sysCaches.contains(e.getKey()))
+                publicProxies.put(e.getKey(), new GridCacheProxyImpl(cache.context(), cache, null));
+        }
 
-            GridCacheStoreManager storeMgr = new GridCacheStoreManager(ctx, sesHolders, cfgStore, cfg);
+        transactions = new IgniteTransactionsImpl(sharedCtx);
 
-            GridCacheContext<?, ?> cacheCtx = new GridCacheContext(
-                ctx,
-                sharedCtx,
-                cfg,
+        if (log.isDebugEnabled())
+            log.debug("Started cache processor.");
+    }
 
-                /*
-                 * Managers in starting order!
-                 * ===========================
-                 */
-                evtMgr,
-                swapMgr,
-                serMgr,
-                storeMgr,
-                evictMgr,
-                qryMgr,
-                contQryMgr,
-                affMgr,
-                dataStructuresMgr,
-                ttlMgr,
-                drMgr,
-                jta);
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void onKernalStart() throws IgniteCheckedException {
+        if (ctx.config().isDaemon())
+            return;
 
-            cacheCtx.defaultAffMapper(dfltAffMapper);
+        if (!getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK)) {
+            for (ClusterNode n : ctx.discovery().remoteNodes())
+                checkCache(n);
+        }
 
-            GridCacheAdapter cache = null;
+        for (Map.Entry<String, GridCacheAdapter<?, ?>> e : caches.entrySet()) {
+            GridCacheAdapter cache = e.getValue();
 
-            switch (cfg.getCacheMode()) {
-                case LOCAL: {
-                    switch (cfg.getAtomicityMode()) {
-                        case TRANSACTIONAL: {
-                            cache = new GridLocalCache(cacheCtx);
+            if (maxPreloadOrder > 0) {
+                CacheConfiguration cfg = cache.configuration();
 
-                            break;
-                        }
-                        case ATOMIC: {
-                            cache = new GridLocalAtomicCache(cacheCtx);
+                int order = cfg.getPreloadOrder();
 
-                            break;
-                        }
+                if (order > 0 && order != maxPreloadOrder && cfg.getCacheMode() != LOCAL) {
+                    GridCompoundFuture<Object, Object> fut = (GridCompoundFuture<Object, Object>)preloadFuts
+                        .get(order);
 
-                        default: {
-                            assert false : "Invalid cache atomicity mode: " + cfg.getAtomicityMode();
-                        }
-                    }
+                    if (fut == null) {
+                        fut = new GridCompoundFuture<>(ctx);
 
-                    break;
-                }
-                case PARTITIONED:
-                case REPLICATED: {
-                    if (GridCacheUtils.isNearEnabled(cfg)) {
-                        switch (cfg.getAtomicityMode()) {
-                            case TRANSACTIONAL: {
-                                cache = new GridNearTransactionalCache(cacheCtx);
-
-                                break;
-                            }
-                            case ATOMIC: {
-                                cache = new GridNearAtomicCache(cacheCtx);
-
-                                break;
-                            }
-
-                            default: {
-                                assert false : "Invalid cache atomicity mode: " + cfg.getAtomicityMode();
-                            }
-                        }
-                    }
-                    else {
-                        switch (cfg.getAtomicityMode()) {
-                            case TRANSACTIONAL: {
-                                cache = GridCacheUtils.isAffinityNode(cfg) ? new GridDhtColocatedCache(cacheCtx) :
-                                    new GridDhtColocatedCache(cacheCtx, new GridNoStorageCacheMap(cacheCtx));
-
-                                break;
-                            }
-                            case ATOMIC: {
-                                cache = GridCacheUtils.isAffinityNode(cfg) ? new GridDhtAtomicCache(cacheCtx) :
-                                    new GridDhtAtomicCache(cacheCtx, new GridNoStorageCacheMap(cacheCtx));
-
-                                break;
-                            }
-
-                            default: {
-                                assert false : "Invalid cache atomicity mode: " + cfg.getAtomicityMode();
-                            }
-                        }
+                        preloadFuts.put(order, fut);
                     }
 
-                    break;
-                }
-
-                default: {
-                    assert false : "Invalid cache mode: " + cfg.getCacheMode();
+                    fut.add(cache.preloader().syncFuture());
                 }
             }
+        }
 
-            cacheCtx.cache(cache);
+        for (IgniteInternalFuture<?> fut : preloadFuts.values())
+            ((GridCompoundFuture<Object, Object>)fut).markInitialized();
 
-            if (caches.containsKey(cfg.getName())) {
-                String cacheName = cfg.getName();
+        for (GridCacheSharedManager<?, ?> mgr : sharedCtx.managers())
+            mgr.onKernalStart();
 
-                if (cacheName != null)
-                    throw new IgniteCheckedException("Duplicate cache name found (check configuration and " +
-                        "assign unique name to each cache): " + cacheName);
-                else
-                    throw new IgniteCheckedException("Default cache has already been configured (check configuration and " +
-                        "assign unique name to each cache).");
+        for (GridCacheAdapter<?, ?> cache : caches.values())
+            onKernalStart(cache);
+
+        // Wait for caches in SYNC preload mode.
+        for (GridCacheAdapter<?, ?> cache : caches.values()) {
+            CacheConfiguration cfg = cache.configuration();
+
+            if (cfg.getPreloadMode() == SYNC) {
+                if (cfg.getCacheMode() == REPLICATED ||
+                    (cfg.getCacheMode() == PARTITIONED && cfg.getPreloadPartitionedDelay() >= 0))
+                    cache.preloader().syncFuture().get();
             }
+        }
 
-            caches.put(cfg.getName(), cache);
+        ctx.portable().onCacheProcessorStarted();
+    }
 
-            if (sysCaches.contains(cfg.getName()))
-                stopSeq.addLast(cache);
-            else
-                stopSeq.addFirst(cache);
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void stop(boolean cancel) throws IgniteCheckedException {
+        if (ctx.config().isDaemon())
+            return;
 
-            startSeq.add(cache);
+        for (GridCacheAdapter<?, ?> cache : stopSeq)
+            stopCache(cache, cancel);
 
-            /*
-             * Create DHT cache.
-             * ================
-             */
-            if (cfg.getCacheMode() != LOCAL && GridCacheUtils.isNearEnabled(cfg)) {
-                /*
-                 * Specifically don't create the following managers
-                 * here and reuse the one from Near cache:
-                 * 1. GridCacheVersionManager
-                 * 2. GridCacheIoManager
-                 * 3. GridCacheDeploymentManager
-                 * 4. GridCacheQueryManager (note, that we start it for DHT cache though).
-                 * 5. CacheContinuousQueryManager (note, that we start it for DHT cache though).
-                 * 6. GridCacheDgcManager
-                 * 7. GridCacheTtlManager.
-                 * ===============================================
-                 */
-                swapMgr = new GridCacheSwapManager(true);
-                evictMgr = new GridCacheEvictionManager();
-                evtMgr = new GridCacheEventManager();
-                drMgr = ctx.createComponent(GridCacheDrManager.class);
+        List<? extends GridCacheSharedManager<?, ?>> mgrs = sharedCtx.managers();
 
-                cacheCtx = new GridCacheContext(
-                    ctx,
-                    sharedCtx,
-                    cfg,
+        for (ListIterator<? extends GridCacheSharedManager<?, ?>> it = mgrs.listIterator(mgrs.size()); it.hasPrevious();) {
+            GridCacheSharedManager<?, ?> mgr = it.previous();
 
-                    /*
-                     * Managers in starting order!
-                     * ===========================
-                     */
-                    evtMgr,
-                    swapMgr,
-                    serMgr,
-                    storeMgr,
-                    evictMgr,
-                    qryMgr,
-                    contQryMgr,
-                    affMgr,
-                    dataStructuresMgr,
-                    ttlMgr,
-                    drMgr,
-                    jta);
-
-                cacheCtx.defaultAffMapper(dfltAffMapper);
-
-                GridDhtCacheAdapter dht = null;
+            mgr.stop(cancel);
+        }
 
-                switch (cfg.getAtomicityMode()) {
-                    case TRANSACTIONAL: {
-                        assert cache instanceof GridNearTransactionalCache;
+        sharedCtx.cleanup();
 
-                        GridNearTransactionalCache near = (GridNearTransactionalCache)cache;
+        if (log.isDebugEnabled())
+            log.debug("Stopped cache processor.");
+    }
 
-                        GridDhtCache dhtCache = !GridCacheUtils.isAffinityNode(cfg) ?
-                            new GridDhtCache(cacheCtx, new GridNoStorageCacheMap(cacheCtx)) :
-                            new GridDhtCache(cacheCtx);
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void onKernalStop(boolean cancel) {
+        if (ctx.config().isDaemon())
+            return;
 
-                        dhtCache.near(near);
+        for (GridCacheAdapter<?, ?> cache : stopSeq)
+            onKernalStop(cache, cancel);
 
-                        near.dht(dhtCache);
+        List<? extends GridCacheSharedManager<?, ?>> sharedMgrs = sharedCtx.managers();
 
-                        dht = dhtCache;
+        for (ListIterator<? extends GridCacheSharedManager<?, ?>> it = sharedMgrs.listIterator(sharedMgrs.size());
+            it.hasPrevious();) {
+            GridCacheSharedManager<?, ?> mgr = it.previous();
 
-                        break;
-                    }
-                    case ATOMIC: {
-                        assert cache instanceof GridNearAtomicCache;
+            mgr.onKernalStop(cancel);
+        }
+    }
 
-                        GridNearAtomicCache near = (GridNearAtomicCache)cache;
+    /**
+     * @param cache Cache to start.
+     * @throws IgniteCheckedException If failed to start cache.
+     */
+    @SuppressWarnings({"TypeMayBeWeakened", "unchecked"})
+    private void startCache(GridCacheAdapter<?, ?> cache) throws IgniteCheckedException {
+        GridCacheContext<?, ?> cacheCtx = cache.context();
 
-                        GridDhtAtomicCache dhtCache = GridCacheUtils.isAffinityNode(cfg) ? new GridDhtAtomicCache(cacheCtx) :
-                            new GridDhtAtomicCache(cacheCtx, new GridNoStorageCacheMap(cacheCtx));
+        CacheConfiguration cfg = cacheCtx.config();
 
-                        dhtCache.near(near);
+        // Start managers.
+        for (GridCacheManager mgr : F.view(cacheCtx.managers(), F.notContains(dhtExcludes(cacheCtx))))
+            mgr.start(cacheCtx);
 
-                        near.dht(dhtCache);
+        cacheCtx.initConflictResolver();
 
-                        dht = dhtCache;
+        if (cfg.getCacheMode() != LOCAL && GridCacheUtils.isNearEnabled(cfg)) {
+            GridCacheContext<?, ?> dhtCtx = cacheCtx.near().dht().context();
 
-                        break;
-                    }
+            // Start DHT managers.
+            for (GridCacheManager mgr : dhtManagers(dhtCtx))
+                mgr.start(dhtCtx);
 
-                    default: {
-                        assert false : "Invalid cache atomicity mode: " + cfg.getAtomicityMode();
-                    }
-                }
+            dhtCtx.initConflictResolver();
 
-                cacheCtx.cache(dht);
-            }
+            // Start DHT cache.
+            dhtCtx.cache().start();
 
-            sharedCtx.addCacheContext(cache.context());
+            if (log.isDebugEnabled())
+                log.debug("Started DHT cache: " + dhtCtx.cache().name());
         }
 
-        // Start shared managers.
-        for (GridCacheSharedManager mgr : sharedCtx.managers())
-            mgr.start(sharedCtx);
+        cacheCtx.cache().start();
 
-        for (GridCacheAdapter<?, ?> cache : startSeq) {
-            GridCacheContext<?, ?> cacheCtx = cache.context();
+        if (log.isInfoEnabled())
+            log.info("Started cache [name=" + cfg.getName() + ", mode=" + cfg.getCacheMode() + ']');
+    }
+
+    /**
+     * @param cache Cache to stop.
+     * @param cancel Cancel flag.
+     */
+    @SuppressWarnings({"TypeMayBeWeakened", "unchecked"})
+    private void stopCache(GridCacheAdapter<?, ?> cache, boolean cancel) {
+        GridCacheContext ctx = cache.context();
 
-            CacheConfiguration cfg = cacheCtx.config();
+        sharedCtx.removeCacheContext(ctx);
 
-            // Start managers.
-            for (GridCacheManager mgr : F.view(cacheCtx.managers(), F.notContains(dhtExcludes(cacheCtx))))
-                mgr.start(cacheCtx);
+        cache.stop();
 
-            cacheCtx.initConflictResolver();
+        if (isNearEnabled(ctx)) {
+            GridDhtCacheAdapter dht = ctx.near().dht();
 
-            if (cfg.getCacheMode() != LOCAL && GridCacheUtils.isNearEnabled(cfg)) {
-                GridCacheContext<?, ?> dhtCtx = cacheCtx.near().dht().context();
+            // Check whether dht cache has been started.
+            if (dht != null) {
+                dht.stop();
 
-                // Start DHT managers.
-                for (GridCacheManager mgr : dhtManagers(dhtCtx))
-                    mgr.start(dhtCtx);
+                GridCacheContext<?, ?> dhtCtx = dht.context();
 
-                dhtCtx.initConflictResolver();
+                List<GridCacheManager> dhtMgrs = dhtManagers(dhtCtx);
 
-                // Start DHT cache.
-                dhtCtx.cache().start();
+                for (ListIterator<GridCacheManager> it = dhtMgrs.listIterator(dhtMgrs.size()); it.hasPrevious();) {
+                    GridCacheManager mgr = it.previous();
 
-                if (log.isDebugEnabled())
-                    log.debug("Started DHT cache: " + dhtCtx.cache().name());
+                    mgr.stop(cancel);
+                }
             }
+        }
+
+        List<GridCacheManager> mgrs = ctx.managers();
+
+        Collection<GridCacheManager> excludes = dhtExcludes(ctx);
 
-            cacheCtx.cache().start();
+        // Reverse order.
+        for (ListIterator<GridCacheManager> it = mgrs.listIterator(mgrs.size()); it.hasPrevious();) {
+            GridCacheManager mgr = it.previous();
 
-            if (log.isInfoEnabled())
-                log.info("Started cache [name=" + cfg.getName() + ", mode=" + cfg.getCacheMode() + ']');
+            if (!excludes.contains(mgr))
+                mgr.stop(cancel);
         }
 
-        for (Map.Entry<String, GridCacheAdapter<?, ?>> e : caches.entrySet()) {
-            GridCacheAdapter cache = e.getValue();
+        U.stopLifecycleAware(log, lifecycleAwares(cache.configuration(), ctx.jta().tmLookup(),
+            ctx.store().configuredStore()));
 
-            proxies.put(e.getKey(), new GridCacheProxyImpl(cache.context(), cache, null));
+        if (log.isInfoEnabled())
+            log.info("Stopped cache: " + cache.name());
 
-            jCacheProxies.put(e.getKey(), new IgniteCacheProxy(cache.context(), cache, null, false));
-        }
+        cleanup(ctx);
+    }
 
-        // Internal caches which should not be returned to user.
-        for (Map.Entry<String, GridCacheAdapter<?, ?>> e : caches.entrySet()) {
-            GridCacheAdapter cache = e.getValue();
+    /**
+     * @param cache Cache.
+     * @throws IgniteCheckedException If failed.
+     */
+    @SuppressWarnings("unchecked")
+    private void onKernalStart(GridCacheAdapter<?, ?> cache) throws IgniteCheckedException {
+        GridCacheContext<?, ?> ctx = cache.context();
 
-            if (!sysCaches.contains(e.getKey()))
-                publicProxies.put(e.getKey(), new GridCacheProxyImpl(cache.context(), cache, null));
+        // Start DHT cache as well.
+        if (isNearEnabled(ctx)) {
+            GridDhtCacheAdapter dht = ctx.near().dht();
+
+            GridCacheContext<?, ?> dhtCtx = dht.context();
+
+            for (GridCacheManager mgr : dhtManagers(dhtCtx))
+                mgr.onKernalStart();
+
+            dht.onKernalStart();
+
+            if (log.isDebugEnabled())
+                log.debug("Executed onKernalStart() callback for DHT cache: " + dht.name());
         }
 
-        transactions = new IgniteTransactionsImpl(sharedCtx);
+        for (GridCacheManager mgr : F.view(ctx.managers(), F0.notContains(dhtExcludes(ctx))))
+            mgr.onKernalStart();
+
+        cache.onKernalStart();
 
         if (log.isDebugEnabled())
-            log.debug("Started cache processor.");
+            log.debug("Executed onKernalStart() callback for cache [name=" + cache.name() + ", mode=" +
+                cache.configuration().getCacheMode() + ']');
     }
 
     /**
-     * Callback invoked when first exchange future for dynamic cache is completed.
-     *
-     * @param startDesc Cache start descriptor.
+     * @param cache Cache to stop.
+     * @param cancel Cancel flag.
      */
-    public void onCacheStartFinished(DynamicCacheDescriptor startDesc) {
-        CacheConfiguration ccfg = startDesc.cacheConfiguration();
+    @SuppressWarnings("unchecked")
+    private void onKernalStop(GridCacheAdapter<?, ?> cache, boolean cancel) {
+        GridCacheContext ctx = cache.context();
 
-        DynamicCacheStartFuture fut = (DynamicCacheStartFuture)pendingStarts.get(ccfg.getName());
+        if (isNearEnabled(ctx)) {
+            GridDhtCacheAdapter dht = ctx.near().dht();
 
-        if (fut != null && fut.startId().equals(startDesc.startId())) {
-            fut.onDone();
+            if (dht != null) {
+                GridCacheContext<?, ?> dhtCtx = dht.context();
 
-            pendingStarts.remove(ccfg.getName(), fut);
+                for (GridCacheManager mgr : dhtManagers(dhtCtx))
+                    mgr.onKernalStop(cancel);
+
+                dht.onKernalStop();
+            }
+        }
+
+        List<GridCacheManager> mgrs = ctx.managers();
+
+        Collection<GridCacheManager> excludes = dhtExcludes(ctx);
+
+        // Reverse order.
+        for (ListIterator<GridCacheManager> it = mgrs.listIterator(mgrs.size()); it.hasPrevious(); ) {
+            GridCacheManager mgr = it.previous();
+
+            if (!excludes.contains(mgr))
+                mgr.onKernalStop(cancel);
         }
+
+        cache.onKernalStop();
     }
 
     /**
-     * Creates shared context.
-     *
-     * @param kernalCtx Kernal context.
+     * @param cfg Cache configuration to use to create cache.
+     * @return Cache context.
+     * @throws IgniteCheckedException If failed to create cache.
+     */
+    @SuppressWarnings( {"unchecked"})
+    private GridCacheContext createCache(CacheConfiguration<?, ?> cfg) throws IgniteCheckedException {
+        CacheStore cfgStore = cfg.getCacheStoreFactory() != null ? cfg.getCacheStoreFactory().create() : null;
+
+        validate(ctx.config(), cfg, cfgStore);
+
+        CacheJtaManagerAdapter jta = JTA.create(cfg.getTransactionManagerLookupClassName() == null);
+
+        jta.createTmLookup(cfg);
+
+        // Skip suggestions for system caches.
+        if (!sysCaches.contains(cfg.getName()))
+            suggestOptimizations(cfg, cfgStore != null);
+
+        List<Object> toPrepare = new ArrayList<>();
+
+        toPrepare.add(jta.tmLookup());
+        toPrepare.add(cfgStore);
+
+        if (cfgStore instanceof GridCacheLoaderWriterStore) {
+            toPrepare.add(((GridCacheLoaderWriterStore)cfgStore).loader());
+            toPrepare.add(((GridCacheLoaderWriterStore)cfgStore).writer());
+        }
+
+        prepare(cfg, toPrepare.toArray(new Object[toPrepare.size()]));
+
+        U.startLifecycleAware(lifecycleAwares(cfg, jta.tmLookup(), cfgStore));
+
+        // Init default key mapper.
+        CacheAffinityKeyMapper dfltAffMapper;
+
+        if (cfg.getAffinityMapper().getClass().equals(GridCacheDefaultAffinityKeyMapper.class))
+            dfltAffMapper = cfg.getAffinityMapper();
+        else {
+            dfltAffMapper = new GridCacheDefaultAffinityKeyMapper();
+
+            prepare(cfg, dfltAffMapper, false);
+        }
+
+        GridCacheAffinityManager affMgr = new GridCacheAffinityManager();
+        GridCacheEventManager evtMgr = new GridCacheEventManager();
+        GridCacheSwapManager swapMgr = new GridCacheSwapManager(cfg.getCacheMode() == LOCAL || !GridCacheUtils.isNearEnabled(cfg));
+        GridCacheEvictionManager evictMgr = new GridCacheEvictionManager();
+        GridCacheQueryManager qryMgr = queryManager(cfg);
+        CacheContinuousQueryManager contQryMgr = new CacheContinuousQueryManager();
+        CacheDataStructuresManager dataStructuresMgr = new CacheDataStructuresManager();
+        GridCacheTtlManager ttlMgr = new GridCacheTtlManager();
+        GridCacheDrManager drMgr = ctx.createComponent(GridCacheDrManager.class);
+        IgniteCacheSerializationManager serMgr = ctx.createComponent(IgniteCacheSerializationManager.class);
+
+        GridCacheStoreManager storeMgr = new GridCacheStoreManager(ctx, sesHolders, cfgStore, cfg);
+
+        GridCacheContext<?, ?> cacheCtx = new GridCacheContext(
+            ctx,
+            sharedCtx,
+            cfg,
+
+                /*
+                 * Managers in starting order!
+                 * ===========================
+                 */
+            evtMgr,
+            swapMgr,
+            serMgr,
+            storeMgr,
+            evictMgr,
+            qryMgr,
+            contQryMgr,
+            affMgr,
+            dataStructuresMgr,
+            ttlMgr,
+            drMgr,
+            jta);
+
+        cacheCtx.defaultAffMapper(dfltAffMapper);
+
+        GridCacheAdapter cache = null;
+
+        switch (cfg.getCacheMode()) {
+            case LOCAL: {
+                switch (cfg.getAtomicityMode()) {
+                    case TRANSACTIONAL: {
+                        cache = new GridLocalCache(cacheCtx);
+
+                        break;
+                    }
+                    case ATOMIC: {
+                        cache = new GridLocalAtomicCache(cacheCtx);
+
+                        break;
+                    }
+
+                    default: {
+                        assert false : "Invalid cache atomicity mode: " + cfg.getAtomicityMode();
+                    }
+                }
+
+                break;
+            }
+            case PARTITIONED:
+            case REPLICATED: {
+                if (GridCacheUtils.isNearEnabled(cfg)) {
+                    switch (cfg.getAtomicityMode()) {
+                        case TRANSACTIONAL: {
+                            cache = new GridNearTransactionalCache(cacheCtx);
+
+                            break;
+                        }
+                        case ATOMIC: {
+                            cache = new GridNearAtomicCache(cacheCtx);
+
+                            break;
+                        }
+
+                        default: {
+                            assert false : "Invalid cache atomicity mode: " + cfg.getAtomicityMode();
+                        }
+                    }
+                }
+                else {
+                    switch (cfg.getAtomicityMode()) {
+                        case TRANSACTIONAL: {
+                            cache = GridCacheUtils.isAffinityNode(cfg) ? new GridDhtColocatedCache(cacheCtx) :
+                                new GridDhtColocatedCache(cacheCtx, new GridNoStorageCacheMap(cacheCtx));
+
+                            break;
+                        }
+                        case ATOMIC: {
+                            cache = GridCacheUtils.isAffinityNode(cfg) ? new GridDhtAtomicCache(cacheCtx) :
+                                new GridDhtAtomicCache(cacheCtx, new GridNoStorageCacheMap(cacheCtx));
+
+                            break;
+                        }
+
+                        default: {
+                            assert false : "Invalid cache atomicity mode: " + cfg.getAtomicityMode();
+                        }
+                    }
+                }
+
+                break;
+            }
+
+            default: {
+                assert false : "Invalid cache mode: " + cfg.getCacheMode();
+            }
+        }
+
+        cacheCtx.cache(cache);
+
+        GridCacheContext<?, ?> ret = cacheCtx;
+
+        /*
+         * Create DHT cache.
+         * ================
+         */
+        if (cfg.getCacheMode() != LOCAL && GridCacheUtils.isNearEnabled(cfg)) {
+            /*
+             * Specifically don't create the following managers
+             * here and reuse the one from Near cache:
+             * 1. GridCacheVersionManager
+             * 2. GridCacheIoManager
+             * 3. GridCacheDeploymentManager
+             * 4. GridCacheQueryManager (note, that we start it for DHT cache though).
+             * 5. CacheContinuousQueryManager (note, that we start it for DHT cache though).
+             * 6. GridCacheDgcManager
+             * 7. GridCacheTtlManager.
+             * ===============================================
+             */
+            swapMgr = new GridCacheSwapManager(true);
+            evictMgr = new GridCacheEvictionManager();
+            evtMgr = new GridCacheEventManager();
+            drMgr = ctx.createComponent(GridCacheDrManager.class);
+
+            cacheCtx = new GridCacheContext(
+                ctx,
+                sharedCtx,
+                cfg,
+
+                    /*
+                     * Managers in starting order!
+                     * ===========================
+                     */
+                evtMgr,
+                swapMgr,
+                serMgr,
+                storeMgr,
+                evictMgr,
+                qryMgr,
+                contQryMgr,
+                affMgr,
+                dataStructuresMgr,
+                ttlMgr,
+                drMgr,
+                jta);
+
+            cacheCtx.defaultAffMapper(dfltAffMapper);
+
+            GridDhtCacheAdapter dht = null;
+
+            switch (cfg.getAtomicityMode()) {
+                case TRANSACTIONAL: {
+                    assert cache instanceof GridNearTransactionalCache;
+
+                    GridNearTransactionalCache near = (GridNearTransactionalCache)cache;
+
+                    GridDhtCache dhtCache = !GridCacheUtils.isAffinityNode(cfg) ?
+                        new GridDhtCache(cacheCtx, new GridNoStorageCacheMap(cacheCtx)) :
+                        new GridDhtCache(cacheCtx);
+
+                    dhtCache.near(near);
+
+                    near.dht(dhtCache);
+
+                    dht = dhtCache;
+
+                    break;
+                }
+                case ATOMIC: {
+                    assert cache instanceof GridNearAtomicCache;
+
+                    GridNearAtomicCache near = (GridNearAtomicCache)cache;
+
+                    GridDhtAtomicCache dhtCache = GridCacheUtils.isAffinityNode(cfg) ? new GridDhtAtomicCache(cacheCtx) :
+                        new GridDhtAtomicCache(cacheCtx, new GridNoStorageCacheMap(cacheCtx));
+
+                    dhtCache.near(near);
+
+                    near.dht(dhtCache);
+
+                    dht = dhtCache;
+
+                    break;
+                }
+
+                default: {
+                    assert false : "Invalid cache atomicity mode: " + cfg.getAtomicityMode();
+                }
+            }
+
+            cacheCtx.cache(dht);
+        }
+
+        return ret;
+    }
+
+    /**
+     * Callback invoked when first exchange future for dynamic cache is completed.
+     *
+     * @param startDesc Cache start descriptor.
+     */
+    public void onCacheStartFinished(DynamicCacheDescriptor startDesc) {
+        CacheConfiguration ccfg = startDesc.cacheConfiguration();
+
+        DynamicCacheStartFuture fut = (DynamicCacheStartFuture)pendingStarts.get(ccfg.getName());
+
+        if (fut != null && fut.startId().equals(startDesc.startId())) {
+            fut.onDone();
+
+            pendingStarts.remove(ccfg.getName(), fut);
+        }
+    }
+
+    /**
+     * Creates shared context.
+     *
+     * @param kernalCtx Kernal context.
      * @return Shared context.
      */
     @SuppressWarnings("unchecked")
@@ -1003,7 +1246,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param nodeFilter Node filter to select nodes on which the cache should be deployed.
      * @return Future that will be completed when cache is deployed.
      */
-    public IgniteInternalFuture<?> startCache(CacheConfiguration ccfg, IgnitePredicate<ClusterNode> nodeFilter) {
+    public IgniteInternalFuture<?> dynamicStartCache(CacheConfiguration ccfg, IgnitePredicate<ClusterNode> nodeFilter) {
         if (nodeFilter == null)
             nodeFilter = F.alwaysTrue();
 
@@ -1048,9 +1291,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param startDesc Cache start descriptor.
      */
     private void onCacheDeploymentRequested(DynamicCacheDescriptor startDesc) {
-        // TODO IGNITE-45 remove debug
-        U.debug(log, "Received start notification: " + startDesc);
-
         CacheConfiguration ccfg = startDesc.cacheConfiguration();
 
         // Check if cache with the same name was concurrently started form different node.
@@ -1367,208 +1607,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param cache Cache.
-     * @throws IgniteCheckedException If failed.
-     */
-    @SuppressWarnings("unchecked")
-    private void onKernalStart(GridCacheAdapter<?, ?> cache) throws IgniteCheckedException {
-        GridCacheContext<?, ?> ctx = cache.context();
-
-        // Start DHT cache as well.
-        if (isNearEnabled(ctx)) {
-            GridDhtCacheAdapter dht = ctx.near().dht();
-
-            GridCacheContext<?, ?> dhtCtx = dht.context();
-
-            for (GridCacheManager mgr : dhtManagers(dhtCtx))
-                mgr.onKernalStart();
-
-            dht.onKernalStart();
-
-            if (log.isDebugEnabled())
-                log.debug("Executed onKernalStart() callback for DHT cache: " + dht.name());
-        }
-
-        for (GridCacheManager mgr : F.view(ctx.managers(), F0.notContains(dhtExcludes(ctx))))
-            mgr.onKernalStart();
-
-        cache.onKernalStart();
-
-        if (log.isDebugEnabled())
-            log.debug("Executed onKernalStart() callback for cache [name=" + cache.name() + ", mode=" +
-                cache.configuration().getCacheMode() + ']');
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public void onKernalStart() throws IgniteCheckedException {
-        if (ctx.config().isDaemon())
-            return;
-
-        if (!getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK)) {
-            for (ClusterNode n : ctx.discovery().remoteNodes())
-                checkCache(n);
-        }
-
-        for (Map.Entry<String, GridCacheAdapter<?, ?>> e : caches.entrySet()) {
-            GridCacheAdapter cache = e.getValue();
-
-            if (maxPreloadOrder > 0) {
-                CacheConfiguration cfg = cache.configuration();
-
-                int order = cfg.getPreloadOrder();
-
-                if (order > 0 && order != maxPreloadOrder && cfg.getCacheMode() != LOCAL) {
-                    GridCompoundFuture<Object, Object> fut = (GridCompoundFuture<Object, Object>)preloadFuts
-                        .get(order);
-
-                    if (fut == null) {
-                        fut = new GridCompoundFuture<>(ctx);
-
-                        preloadFuts.put(order, fut);
-                    }
-
-                    fut.add(cache.preloader().syncFuture());
-                }
-            }
-        }
-
-        for (IgniteInternalFuture<?> fut : preloadFuts.values())
-            ((GridCompoundFuture<Object, Object>)fut).markInitialized();
-
-        for (GridCacheSharedManager<?, ?> mgr : sharedCtx.managers())
-            mgr.onKernalStart();
-
-        for (GridCacheAdapter<?, ?> cache : caches.values())
-            onKernalStart(cache);
-
-        // Wait for caches in SYNC preload mode.
-        for (GridCacheAdapter<?, ?> cache : caches.values()) {
-            CacheConfiguration cfg = cache.configuration();
-
-            if (cfg.getPreloadMode() == SYNC) {
-                if (cfg.getCacheMode() == REPLICATED ||
-                    (cfg.getCacheMode() == PARTITIONED && cfg.getPreloadPartitionedDelay() >= 0))
-                    cache.preloader().syncFuture().get();
-            }
-        }
-
-        ctx.portable().onCacheProcessorStarted();
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public void onKernalStop(boolean cancel) {
-        if (ctx.config().isDaemon())
-            return;
-
-        for (GridCacheAdapter<?, ?> cache : stopSeq) {
-            GridCacheContext ctx = cache.context();
-
-            if (isNearEnabled(ctx)) {
-                GridDhtCacheAdapter dht = ctx.near().dht();
-
-                if (dht != null) {
-                    GridCacheContext<?, ?> dhtCtx = dht.context();
-
-                    for (GridCacheManager mgr : dhtManagers(dhtCtx))
-                        mgr.onKernalStop(cancel);
-
-                    dht.onKernalStop();
-                }
-            }
-
-            List<GridCacheManager> mgrs = ctx.managers();
-
-            Collection<GridCacheManager> excludes = dhtExcludes(ctx);
-
-            // Reverse order.
-            for (ListIterator<GridCacheManager> it = mgrs.listIterator(mgrs.size()); it.hasPrevious(); ) {
-                GridCacheManager mgr = it.previous();
-
-                if (!excludes.contains(mgr))
-                    mgr.onKernalStop(cancel);
-            }
-
-            cache.onKernalStop();
-        }
-
-        List<? extends GridCacheSharedManager<?, ?>> sharedMgrs = sharedCtx.managers();
-
-        for (ListIterator<? extends GridCacheSharedManager<?, ?>> it = sharedMgrs.listIterator(sharedMgrs.size());
-            it.hasPrevious();) {
-            GridCacheSharedManager<?, ?> mgr = it.previous();
-
-            mgr.onKernalStop(cancel);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public void stop(boolean cancel) throws IgniteCheckedException {
-        if (ctx.config().isDaemon())
-            return;
-
-        for (GridCacheAdapter<?, ?> cache : stopSeq) {
-            cache.stop();
-
-            GridCacheContext ctx = cache.context();
-
-            if (isNearEnabled(ctx)) {
-                GridDhtCacheAdapter dht = ctx.near().dht();
-
-                // Check whether dht cache has been started.
-                if (dht != null) {
-                    dht.stop();
-
-                    GridCacheContext<?, ?> dhtCtx = dht.context();
-
-                    List<GridCacheManager> dhtMgrs = dhtManagers(dhtCtx);
-
-                    for (ListIterator<GridCacheManager> it = dhtMgrs.listIterator(dhtMgrs.size()); it.hasPrevious();) {
-                        GridCacheManager mgr = it.previous();
-
-                        mgr.stop(cancel);
-                    }
-                }
-            }
-
-            List<GridCacheManager> mgrs = ctx.managers();
-
-            Collection<GridCacheManager> excludes = dhtExcludes(ctx);
-
-            // Reverse order.
-            for (ListIterator<GridCacheManager> it = mgrs.listIterator(mgrs.size()); it.hasPrevious();) {
-                GridCacheManager mgr = it.previous();
-
-                if (!excludes.contains(mgr))
-                    mgr.stop(cancel);
-            }
-
-            U.stopLifecycleAware(log, lifecycleAwares(cache.configuration(), ctx.jta().tmLookup(),
-                ctx.store().configuredStore()));
-
-            if (log.isInfoEnabled())
-                log.info("Stopped cache: " + cache.name());
-
-            cleanup(ctx);
-        }
-
-        List<? extends GridCacheSharedManager<?, ?>> mgrs = sharedCtx.managers();
-
-        for (ListIterator<? extends GridCacheSharedManager<?, ?>> it = mgrs.listIterator(mgrs.size()); it.hasPrevious();) {
-            GridCacheSharedManager<?, ?> mgr = it.previous();
-
-            mgr.stop(cancel);
-        }
-
-        sharedCtx.cleanup();
-
-        if (log.isDebugEnabled())
-            log.debug("Stopped cache processor.");
-    }
-
-    /**
      * Gets preload finish future for preload-ordered cache with given order. I.e. will get compound preload future
      * with maximum order less than {@code order}.
      *

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/501bd5c3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index de9ec0e..e133a17 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -36,6 +36,7 @@ import org.apache.ignite.marshaller.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
+import java.util.concurrent.*;
 
 import static org.apache.ignite.internal.processors.cache.CacheFlag.*;
 
@@ -69,7 +70,7 @@ public class GridCacheSharedContext<K, V> {
     private GridCacheDeploymentManager<K, V> depMgr;
 
     /** Cache contexts map. */
-    private Map<Integer, GridCacheContext<K, V>> ctxMap;
+    private ConcurrentMap<Integer, GridCacheContext<K, V>> ctxMap;
 
     /** Tx metrics. */
     private volatile TransactionMetricsAdapter txMetrics;
@@ -101,7 +102,7 @@ public class GridCacheSharedContext<K, V> {
 
         txMetrics = new TransactionMetricsAdapter();
 
-        ctxMap = new HashMap<>();
+        ctxMap = new ConcurrentHashMap<>();
     }
 
     /**
@@ -116,7 +117,7 @@ public class GridCacheSharedContext<K, V> {
     /**
      * Adds cache context to shared cache context.
      *
-     * @param cacheCtx Cache context.
+     * @param cacheCtx Cache context to add.
      */
     @SuppressWarnings("unchecked")
     public void addCacheContext(GridCacheContext cacheCtx) throws IgniteCheckedException {
@@ -132,6 +133,13 @@ public class GridCacheSharedContext<K, V> {
     }
 
     /**
+     * @param cacheCtx Cache context to remove.
+     */
+    public void removeCacheContext(GridCacheContext cacheCtx) {
+        ctxMap.remove(cacheCtx.cacheId(), cacheCtx);
+    }
+
+    /**
      * @return List of shared context managers in starting order.
      */
     public List<GridCacheSharedManager<K, V>> managers() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/501bd5c3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
index dddf4a2..5d515e3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
@@ -65,7 +65,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
 
                 ccfg.setName("TestCacheName");
 
-                futs.add(kernal.context().cache().startCache(ccfg, F.<ClusterNode>alwaysTrue()));
+                futs.add(kernal.context().cache().dynamicStartCache(ccfg, F.<ClusterNode>alwaysTrue()));
 
                 return null;
             }
@@ -109,7 +109,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
 
                 IgniteKernal kernal = (IgniteKernal)grid(ThreadLocalRandom.current().nextInt(nodeCount()));
 
-                futs.add(kernal.context().cache().startCache(ccfg, F.<ClusterNode>alwaysTrue()));
+                futs.add(kernal.context().cache().dynamicStartCache(ccfg, F.<ClusterNode>alwaysTrue()));
 
                 return null;
             }


[41/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-334' into sprint-2

Posted by ag...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-334' into sprint-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/e5e2a8e1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/e5e2a8e1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/e5e2a8e1

Branch: refs/heads/ignite-45
Commit: e5e2a8e1f1881abf80420d4bf1b64d2775f6cf29
Parents: b81c6a2 c577685
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Mar 3 11:45:03 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Mar 3 11:45:03 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/configuration/IgniteConfiguration.java   | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[38/50] incubator-ignite git commit: #ignite-237: change client test.

Posted by ag...@apache.org.
#ignite-237: change client test.


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

Branch: refs/heads/ignite-45
Commit: f144a892e9747b297610f7d9d19c44f362dd4b1f
Parents: 9efcd80
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Mar 2 19:29:55 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Mar 2 19:29:55 2015 +0300

----------------------------------------------------------------------
 .../client/integration/ClientAbstractMultiNodeSelfTest.java        | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f144a892/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java
index e73f06d..9257b71 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java
@@ -403,8 +403,6 @@ public abstract class ClientAbstractMultiNodeSelfTest extends GridCommonAbstract
 
             if (affinity(cache).isPrimaryOrBackup(g.cluster().localNode(), key))
                 assertEquals("zzz", cache.localPeek(key, CachePeekMode.ONHEAP));
-            else
-                assertNull(cache.localPeek(key, CachePeekMode.ONHEAP));
         }
     }
 


[10/50] incubator-ignite git commit: #ignite-237: move CacheConsistentHashAffinityFunction to org.gridgain.benchmarks.risk.affinity.

Posted by ag...@apache.org.
#ignite-237: move CacheConsistentHashAffinityFunction to org.gridgain.benchmarks.risk.affinity.


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

Branch: refs/heads/ignite-45
Commit: d07931d2ddd8d4798c1ca2b7bc9620036ff59ae9
Parents: 87586da
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Feb 26 18:23:27 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Feb 26 18:23:27 2015 +0300

----------------------------------------------------------------------
 .../ClientAbstractMultiThreadedSelfTest.java    |   4 +-
 .../impl/ClientPartitionAffinitySelfTest.java   |  21 +-
 .../ClientAbstractMultiNodeSelfTest.java        |   4 +-
 .../integration/ClientAbstractSelfTest.java     |   8 -
 .../src/test/resources/spring-server-node.xml   |   2 +-
 .../test/resources/spring-server-ssl-node.xml   |   2 +-
 .../CacheConsistentHashAffinityFunction.java    | 703 -------------------
 .../cache/affinity/consistenthash/package.html  |  24 -
 .../processors/cache/GridCacheAttributes.java   |   9 -
 .../processors/cache/GridCacheProcessor.java    |  12 +-
 .../top/GridTopologyCommandHandler.java         |   7 -
 .../cache/VisorCacheAffinityConfiguration.java  |   7 -
 .../optimized/optimized-classnames.properties   |  12 +-
 .../CacheConsistentHashAffinityFunction.java    | 703 +++++++++++++++++++
 .../GridCachePartitionedAffinitySpreadTest.java |  16 +-
 .../GridCachePartitionedAffinitySelfTest.java   |  53 +-
 16 files changed, 731 insertions(+), 856 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d07931d2/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientAbstractMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientAbstractMultiThreadedSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientAbstractMultiThreadedSelfTest.java
index 6264276..4ab2679 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientAbstractMultiThreadedSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientAbstractMultiThreadedSelfTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.client;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.configuration.*;
@@ -193,7 +193,7 @@ public abstract class ClientAbstractMultiThreadedSelfTest extends GridCommonAbst
     private CacheConfiguration cacheConfiguration(@Nullable String cacheName) throws Exception {
         CacheConfiguration cfg = defaultCacheConfiguration();
 
-        cfg.setAffinity(new CacheConsistentHashAffinityFunction());
+        cfg.setAffinity(new CacheRendezvousAffinityFunction());
 
         cfg.setDistributionMode(NEAR_PARTITIONED);
         cfg.setAtomicityMode(TRANSACTIONAL);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d07931d2/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
index 3a45615..444a84c 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.client.impl;
 
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.client.*;
 import org.apache.ignite.internal.processors.affinity.*;
@@ -28,8 +28,6 @@ import org.apache.ignite.testframework.junits.common.*;
 
 import java.util.*;
 
-import static org.apache.ignite.cache.affinity.consistenthash.CacheConsistentHashAffinityFunction.*;
-
 /**
  * Client's partitioned affinity tests.
  */
@@ -285,7 +283,7 @@ public class ClientPartitionAffinitySelfTest extends GridCommonAbstractTest {
 
         aff.setHashIdResolver(HASH_ID_RSLVR);
 
-        CacheConsistentHashAffinityFunction srvAff = new CacheConsistentHashAffinityFunction();
+        CacheRendezvousAffinityFunction srvAff = new CacheRendezvousAffinityFunction();
 
         getTestResources().inject(srvAff);
 
@@ -324,7 +322,7 @@ public class ClientPartitionAffinitySelfTest extends GridCommonAbstractTest {
                 .replicaCount(replicaCnt)
                 .build());
 
-            ClusterNode srvNode = new TestRichNode(nodeId, replicaCnt);
+            ClusterNode srvNode = new TestRichNode(nodeId);
 
             srvNodes.add(srvNode);
         }
@@ -366,28 +364,22 @@ public class ClientPartitionAffinitySelfTest extends GridCommonAbstractTest {
          */
         private final UUID nodeId;
 
-        /**
-         * Partitioned affinity replicas count.
-         */
-        private final Integer replicaCnt;
 
         /**
          * Externalizable class requires public no-arg constructor.
          */
         @SuppressWarnings("UnusedDeclaration")
         public TestRichNode() {
-            this(UUID.randomUUID(), DFLT_REPLICA_COUNT);
+            this(UUID.randomUUID());
         }
 
         /**
          * Constructs rich node stub to use in emulated server topology.
          *
          * @param nodeId Node id.
-         * @param replicaCnt Partitioned affinity replicas count.
          */
-        private TestRichNode(UUID nodeId, int replicaCnt) {
+        private TestRichNode(UUID nodeId) {
             this.nodeId = nodeId;
-            this.replicaCnt = replicaCnt;
         }
 
         /** {@inheritDoc} */
@@ -397,9 +389,6 @@ public class ClientPartitionAffinitySelfTest extends GridCommonAbstractTest {
 
         /** {@inheritDoc} */
         @Override public <T> T attribute(String name) {
-            if (DFLT_REPLICA_COUNT_ATTR_NAME.equals(name))
-                return (T)replicaCnt;
-
             return super.attribute(name);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d07931d2/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java
index 3da6080..0eb6e16 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.client.integration;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.configuration.*;
@@ -196,7 +196,7 @@ public abstract class ClientAbstractMultiNodeSelfTest extends GridCommonAbstract
 
         cfg.setWriteSynchronizationMode(REPLICATED_ASYNC_CACHE_NAME.equals(cacheName) ? FULL_ASYNC : FULL_SYNC);
 
-        cfg.setAffinity(new CacheConsistentHashAffinityFunction());
+        cfg.setAffinity(new CacheRendezvousAffinityFunction());
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d07931d2/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractSelfTest.java
index e55b09d..8011945 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractSelfTest.java
@@ -21,7 +21,6 @@ import junit.framework.*;
 import net.sf.json.*;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.configuration.*;
@@ -1094,13 +1093,6 @@ public abstract class ClientAbstractSelfTest extends GridCommonAbstractTest {
         assertEquals(grid().localNode().id(), node.nodeId());
         assertEquals(4, node.caches().size());
 
-        Integer replica = grid().localNode().attribute(CacheConsistentHashAffinityFunction.DFLT_REPLICA_COUNT_ATTR_NAME);
-
-        if (replica == null)
-            replica = CacheConsistentHashAffinityFunction.DFLT_REPLICA_COUNT;
-
-        assertEquals((int)replica, node.replicaCount());
-
         Map<String, GridClientCacheMode> caches = node.caches();
 
         for (Map.Entry<String, GridClientCacheMode> e : caches.entrySet()) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d07931d2/modules/clients/src/test/resources/spring-server-node.xml
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/resources/spring-server-node.xml b/modules/clients/src/test/resources/spring-server-node.xml
index 3b19f3d..3496e37 100644
--- a/modules/clients/src/test/resources/spring-server-node.xml
+++ b/modules/clients/src/test/resources/spring-server-node.xml
@@ -118,7 +118,7 @@
                     <property name="writeSynchronizationMode" value="FULL_SYNC"/>
 
                     <property name="affinity">
-                        <bean class="org.apache.ignite.cache.affinity.consistenthash.CacheConsistentHashAffinityFunction">
+                        <bean class="org.apache.ignite.cache.affinity.rendezvous.CacheRendezvousAffinityFunction">
                             <constructor-arg value="1"/>
                         </bean>
                     </property>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d07931d2/modules/clients/src/test/resources/spring-server-ssl-node.xml
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/resources/spring-server-ssl-node.xml b/modules/clients/src/test/resources/spring-server-ssl-node.xml
index 3989ac6..c56f25a 100644
--- a/modules/clients/src/test/resources/spring-server-ssl-node.xml
+++ b/modules/clients/src/test/resources/spring-server-ssl-node.xml
@@ -119,7 +119,7 @@
                     <property name="writeSynchronizationMode" value="FULL_SYNC"/>
 
                     <property name="affinity">
-                        <bean class="org.apache.ignite.cache.affinity.consistenthash.CacheConsistentHashAffinityFunction">
+                        <bean class="org.apache.ignite.cache.affinity.rendezvous.CacheRendezvousAffinityFunction">
                             <constructor-arg value="1"/>
                         </bean>
                     </property>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d07931d2/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/CacheConsistentHashAffinityFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/CacheConsistentHashAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/CacheConsistentHashAffinityFunction.java
deleted file mode 100644
index 993205e..0000000
--- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/CacheConsistentHashAffinityFunction.java
+++ /dev/null
@@ -1,703 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.affinity.consistenthash;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.tostring.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.resources.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-/**
- * Affinity function for partitioned cache. This function supports the following
- * configuration:
- * <ul>
- * <li>
- *      {@code backups} - Use this flag to control how many back up nodes will be
- *      assigned to every key. The default value is {@code 0}.
- * </li>
- * <li>
- *      {@code replicas} - Generally the more replicas a node gets, the more key assignments
- *      it will receive. You can configure different number of replicas for a node by
- *      setting user attribute with name {@link #getReplicaCountAttributeName()} to some
- *      number. Default value is {@code 512} defined by {@link #DFLT_REPLICA_COUNT} constant.
- * </li>
- * <li>
- *      {@code backupFilter} - Optional filter for back up nodes. If provided, then only
- *      nodes that pass this filter will be selected as backup nodes. If not provided, then
- *      primary and backup nodes will be selected out of all nodes available for this cache.
- * </li>
- * </ul>
- * <p>
- * Cache affinity can be configured for individual caches via {@link org.apache.ignite.configuration.CacheConfiguration#getAffinity()} method.
- */
-@Deprecated
-public class CacheConsistentHashAffinityFunction implements CacheAffinityFunction {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Flag to enable/disable consistency check (for internal use only). */
-    private static final boolean AFFINITY_CONSISTENCY_CHECK = Boolean.getBoolean("IGNITE_AFFINITY_CONSISTENCY_CHECK");
-
-    /** Default number of partitions. */
-    public static final int DFLT_PARTITION_COUNT = 10000;
-
-    /** Default replica count for partitioned caches. */
-    public static final int DFLT_REPLICA_COUNT = 128;
-
-    /**
-     * Name of node attribute to specify number of replicas for a node.
-     * Default value is {@code gg:affinity:node:replicas}.
-     */
-    public static final String DFLT_REPLICA_COUNT_ATTR_NAME = "gg:affinity:node:replicas";
-
-    /** Node hash. */
-    private transient GridConsistentHash<NodeInfo> nodeHash;
-
-    /** Total number of partitions. */
-    private int parts = DFLT_PARTITION_COUNT;
-
-    /** */
-    private int replicas = DFLT_REPLICA_COUNT;
-
-    /** */
-    private String attrName = DFLT_REPLICA_COUNT_ATTR_NAME;
-
-    /** */
-    private boolean exclNeighbors;
-
-    /**
-     * Optional backup filter. First node passed to this filter is primary node,
-     * and second node is a node being tested.
-     */
-    private IgniteBiPredicate<ClusterNode, ClusterNode> backupFilter;
-
-    /** */
-    private CacheAffinityNodeHashResolver hashIdRslvr = new CacheAffinityNodeAddressHashResolver();
-
-    /** Injected grid. */
-    @IgniteInstanceResource
-    private Ignite ignite;
-
-    /** Injected cache name. */
-    @CacheNameResource
-    private String cacheName;
-
-    /** Injected logger. */
-    @LoggerResource
-    private IgniteLogger log;
-
-    /** Initialization flag. */
-    @SuppressWarnings("TransientFieldNotInitialized")
-    private transient AtomicBoolean init = new AtomicBoolean();
-
-    /** Latch for initializing. */
-    @SuppressWarnings({"TransientFieldNotInitialized"})
-    private transient CountDownLatch initLatch = new CountDownLatch(1);
-
-    /** Nodes IDs. */
-    @GridToStringInclude
-    @SuppressWarnings({"TransientFieldNotInitialized"})
-    private transient ConcurrentMap<UUID, NodeInfo> addedNodes = new ConcurrentHashMap<>();
-
-    /** Optional backup filter. */
-    @GridToStringExclude
-    private final IgniteBiPredicate<NodeInfo, NodeInfo> backupIdFilter = new IgniteBiPredicate<NodeInfo, NodeInfo>() {
-        @Override public boolean apply(NodeInfo primaryNodeInfo, NodeInfo nodeInfo) {
-            return backupFilter == null || backupFilter.apply(primaryNodeInfo.node(), nodeInfo.node());
-        }
-    };
-
-    /** Map of neighbors. */
-    @SuppressWarnings("TransientFieldNotInitialized")
-    private transient ConcurrentMap<UUID, Collection<UUID>> neighbors =
-        new ConcurrentHashMap8<>();
-
-    /**
-     * Empty constructor with all defaults.
-     */
-    public CacheConsistentHashAffinityFunction() {
-        // No-op.
-    }
-
-    /**
-     * Initializes affinity with flag to exclude same-host-neighbors from being backups of each other
-     * and specified number of backups.
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code #getBackupFilter()} is set.
-     *
-     * @param exclNeighbors {@code True} if nodes residing on the same host may not act as backups
-     *      of each other.
-     */
-    public CacheConsistentHashAffinityFunction(boolean exclNeighbors) {
-        this.exclNeighbors = exclNeighbors;
-    }
-
-    /**
-     * Initializes affinity with flag to exclude same-host-neighbors from being backups of each other,
-     * and specified number of backups and partitions.
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code #getBackupFilter()} is set.
-     *
-     * @param exclNeighbors {@code True} if nodes residing on the same host may not act as backups
-     *      of each other.
-     * @param parts Total number of partitions.
-     */
-    public CacheConsistentHashAffinityFunction(boolean exclNeighbors, int parts) {
-        A.ensure(parts != 0, "parts != 0");
-
-        this.exclNeighbors = exclNeighbors;
-        this.parts = parts;
-    }
-
-    /**
-     * Initializes optional counts for replicas and backups.
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code backupFilter} is set.
-     *
-     * @param parts Total number of partitions.
-     * @param backupFilter Optional back up filter for nodes. If provided, backups will be selected
-     *      from all nodes that pass this filter. First argument for this filter is primary node, and second
-     *      argument is node being tested.
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code backupFilter} is set.
-     */
-    public CacheConsistentHashAffinityFunction(int parts,
-                                               @Nullable IgniteBiPredicate<ClusterNode, ClusterNode> backupFilter) {
-        A.ensure(parts != 0, "parts != 0");
-
-        this.parts = parts;
-        this.backupFilter = backupFilter;
-    }
-
-    /**
-     * Gets default count of virtual replicas in consistent hash ring.
-     * <p>
-     * To determine node replicas, node attribute with {@link #getReplicaCountAttributeName()}
-     * name will be checked first. If it is absent, then this value will be used.
-     *
-     * @return Count of virtual replicas in consistent hash ring.
-     */
-    public int getDefaultReplicas() {
-        return replicas;
-    }
-
-    /**
-     * Sets default count of virtual replicas in consistent hash ring.
-     * <p>
-     * To determine node replicas, node attribute with {@link #getReplicaCountAttributeName} name
-     * will be checked first. If it is absent, then this value will be used.
-     *
-     * @param replicas Count of virtual replicas in consistent hash ring.s
-     */
-    public void setDefaultReplicas(int replicas) {
-        this.replicas = replicas;
-    }
-
-    /**
-     * Gets total number of key partitions. To ensure that all partitions are
-     * equally distributed across all nodes, please make sure that this
-     * number is significantly larger than a number of nodes. Also, partition
-     * size should be relatively small. Try to avoid having partitions with more
-     * than quarter million keys.
-     * <p>
-     * Note that for fully replicated caches this method should always
-     * return {@code 1}.
-     *
-     * @return Total partition count.
-     */
-    public int getPartitions() {
-        return parts;
-    }
-
-    /**
-     * Sets total number of partitions.
-     *
-     * @param parts Total number of partitions.
-     */
-    public void setPartitions(int parts) {
-        this.parts = parts;
-    }
-
-    /**
-     * Gets hash ID resolver for nodes. This resolver is used to provide
-     * alternate hash ID, other than node ID.
-     * <p>
-     * Node IDs constantly change when nodes get restarted, which causes them to
-     * be placed on different locations in the hash ring, and hence causing
-     * repartitioning. Providing an alternate hash ID, which survives node restarts,
-     * puts node on the same location on the hash ring, hence minimizing required
-     * repartitioning.
-     *
-     * @return Hash ID resolver.
-     */
-    public CacheAffinityNodeHashResolver getHashIdResolver() {
-        return hashIdRslvr;
-    }
-
-    /**
-     * Sets hash ID resolver for nodes. This resolver is used to provide
-     * alternate hash ID, other than node ID.
-     * <p>
-     * Node IDs constantly change when nodes get restarted, which causes them to
-     * be placed on different locations in the hash ring, and hence causing
-     * repartitioning. Providing an alternate hash ID, which survives node restarts,
-     * puts node on the same location on the hash ring, hence minimizing required
-     * repartitioning.
-     *
-     * @param hashIdRslvr Hash ID resolver.
-     */
-    public void setHashIdResolver(CacheAffinityNodeHashResolver hashIdRslvr) {
-        this.hashIdRslvr = hashIdRslvr;
-    }
-
-    /**
-     * Gets optional backup filter. If not {@code null}, backups will be selected
-     * from all nodes that pass this filter. First node passed to this filter is primary node,
-     * and second node is a node being tested.
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code backupFilter} is set.
-     *
-     * @return Optional backup filter.
-     */
-    @Nullable public IgniteBiPredicate<ClusterNode, ClusterNode> getBackupFilter() {
-        return backupFilter;
-    }
-
-    /**
-     * Sets optional backup filter. If provided, then backups will be selected from all
-     * nodes that pass this filter. First node being passed to this filter is primary node,
-     * and second node is a node being tested.
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code backupFilter} is set.
-     *
-     * @param backupFilter Optional backup filter.
-     */
-    public void setBackupFilter(@Nullable IgniteBiPredicate<ClusterNode, ClusterNode> backupFilter) {
-        this.backupFilter = backupFilter;
-    }
-
-    /**
-     * Gets optional attribute name for replica count. If not provided, the
-     * default is {@link #DFLT_REPLICA_COUNT_ATTR_NAME}.
-     *
-     * @return User attribute name for replica count for a node.
-     */
-    public String getReplicaCountAttributeName() {
-        return attrName;
-    }
-
-    /**
-     * Sets optional attribute name for replica count. If not provided, the
-     * default is {@link #DFLT_REPLICA_COUNT_ATTR_NAME}.
-     *
-     * @param attrName User attribute name for replica count for a node.
-     */
-    public void setReplicaCountAttributeName(String attrName) {
-        this.attrName = attrName;
-    }
-
-    /**
-     * Checks flag to exclude same-host-neighbors from being backups of each other (default is {@code false}).
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code #getBackupFilter()} is set.
-     *
-     * @return {@code True} if nodes residing on the same host may not act as backups of each other.
-     */
-    public boolean isExcludeNeighbors() {
-        return exclNeighbors;
-    }
-
-    /**
-     * Sets flag to exclude same-host-neighbors from being backups of each other (default is {@code false}).
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code #getBackupFilter()} is set.
-     *
-     * @param exclNeighbors {@code True} if nodes residing on the same host may not act as backups of each other.
-     */
-    public void setExcludeNeighbors(boolean exclNeighbors) {
-        this.exclNeighbors = exclNeighbors;
-    }
-
-    /**
-     * Gets neighbors for a node.
-     *
-     * @param node Node.
-     * @return Neighbors.
-     */
-    private Collection<UUID> neighbors(final ClusterNode node) {
-        Collection<UUID> ns = neighbors.get(node.id());
-
-        if (ns == null) {
-            Collection<ClusterNode> nodes = ignite.cluster().forHost(node).nodes();
-
-            ns = F.addIfAbsent(neighbors, node.id(), new ArrayList<>(F.nodeIds(nodes)));
-        }
-
-        return ns;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public List<List<ClusterNode>> assignPartitions(CacheAffinityFunctionContext ctx) {
-        List<List<ClusterNode>> res = new ArrayList<>(parts);
-
-        Collection<ClusterNode> topSnapshot = ctx.currentTopologySnapshot();
-
-        for (int part = 0; part < parts; part++) {
-            res.add(F.isEmpty(topSnapshot) ?
-                Collections.<ClusterNode>emptyList() :
-                // Wrap affinity nodes with unmodifiable list since unmodifiable generic collection
-                // doesn't provide equals and hashCode implementations.
-                U.sealList(nodes(part, topSnapshot, ctx.backups())));
-        }
-
-        return res;
-    }
-
-    /**
-     * Assigns nodes to one partition.
-     *
-     * @param part Partition to assign nodes for.
-     * @param nodes Cache topology nodes.
-     * @return Assigned nodes, first node is primary, others are backups.
-     */
-    public Collection<ClusterNode> nodes(int part, Collection<ClusterNode> nodes, int backups) {
-        if (nodes == null)
-            return Collections.emptyList();
-
-        int nodesSize = nodes.size();
-
-        if (nodesSize == 0)
-            return Collections.emptyList();
-
-        if (nodesSize == 1) // Minor optimization.
-            return nodes;
-
-        initialize();
-
-        final Map<NodeInfo, ClusterNode> lookup = new GridLeanMap<>(nodesSize);
-
-        // Store nodes in map for fast lookup.
-        for (ClusterNode n : nodes)
-            // Add nodes into hash circle, if absent.
-            lookup.put(resolveNodeInfo(n), n);
-
-        Collection<NodeInfo> selected;
-
-        if (backupFilter != null) {
-            final IgnitePredicate<NodeInfo> p = new P1<NodeInfo>() {
-                @Override public boolean apply(NodeInfo id) {
-                    return lookup.containsKey(id);
-                }
-            };
-
-            final NodeInfo primaryId = nodeHash.node(part, p);
-
-            IgnitePredicate<NodeInfo> backupPrimaryIdFilter = new IgnitePredicate<NodeInfo>() {
-                @Override public boolean apply(NodeInfo node) {
-                    return backupIdFilter.apply(primaryId, node);
-                }
-            };
-
-            Collection<NodeInfo> backupIds = nodeHash.nodes(part, backups, p, backupPrimaryIdFilter);
-
-            if (F.isEmpty(backupIds) && primaryId != null) {
-                ClusterNode n = lookup.get(primaryId);
-
-                assert n != null;
-
-                return Collections.singletonList(n);
-            }
-
-            selected = primaryId != null ? F.concat(false, primaryId, backupIds) : backupIds;
-        }
-        else {
-            if (!exclNeighbors) {
-                selected = nodeHash.nodes(part, backups == Integer.MAX_VALUE ? backups : backups + 1, new P1<NodeInfo>() {
-                    @Override public boolean apply(NodeInfo id) {
-                        return lookup.containsKey(id);
-                    }
-                });
-
-                if (selected.size() == 1) {
-                    NodeInfo id = F.first(selected);
-
-                    assert id != null : "Node ID cannot be null in affinity node ID collection: " + selected;
-
-                    ClusterNode n = lookup.get(id);
-
-                    assert n != null;
-
-                    return Collections.singletonList(n);
-                }
-            }
-            else {
-                int primaryAndBackups = backups + 1;
-
-                selected = new ArrayList<>(primaryAndBackups);
-
-                final Collection<NodeInfo> selected0 = selected;
-
-                List<NodeInfo> ids = nodeHash.nodes(part, primaryAndBackups, new P1<NodeInfo>() {
-                    @Override public boolean apply(NodeInfo id) {
-                        ClusterNode n = lookup.get(id);
-
-                        if (n == null)
-                            return false;
-
-                        Collection<UUID> neighbors = neighbors(n);
-
-                        for (NodeInfo id0 : selected0) {
-                            ClusterNode n0 = lookup.get(id0);
-
-                            if (n0 == null)
-                                return false;
-
-                            Collection<UUID> neighbors0 = neighbors(n0);
-
-                            if (F.containsAny(neighbors0, neighbors))
-                                return false;
-                        }
-
-                        selected0.add(id);
-
-                        return true;
-                    }
-                });
-
-                if (AFFINITY_CONSISTENCY_CHECK)
-                    assert F.eqOrdered(ids, selected);
-            }
-        }
-
-        Collection<ClusterNode> ret = new ArrayList<>(selected.size());
-
-        for (NodeInfo id : selected) {
-            ClusterNode n = lookup.get(id);
-
-            assert n != null;
-
-            ret.add(n);
-        }
-
-        return ret;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int partition(Object key) {
-        initialize();
-
-        return U.safeAbs(key.hashCode() % parts);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int partitions() {
-        initialize();
-
-        return parts;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void reset() {
-        addedNodes = new ConcurrentHashMap<>();
-        neighbors = new ConcurrentHashMap8<>();
-
-        initLatch = new CountDownLatch(1);
-
-        init = new AtomicBoolean();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeNode(UUID nodeId) {
-        NodeInfo info = addedNodes.remove(nodeId);
-
-        if (info == null)
-            return;
-
-        nodeHash.removeNode(info);
-
-        neighbors.clear();
-    }
-
-    /**
-     * Resolve node info for specified node.
-     * Add node to hash circle if this is the first node invocation.
-     *
-     * @param n Node to get info for.
-     * @return Node info.
-     */
-    private NodeInfo resolveNodeInfo(ClusterNode n) {
-        UUID nodeId = n.id();
-        NodeInfo nodeInfo = addedNodes.get(nodeId);
-
-        if (nodeInfo != null)
-            return nodeInfo;
-
-        assert hashIdRslvr != null;
-
-        nodeInfo = new NodeInfo(nodeId, hashIdRslvr.resolve(n), n);
-
-        neighbors.clear();
-
-        nodeHash.addNode(nodeInfo, replicas(n));
-
-        addedNodes.put(nodeId, nodeInfo);
-
-        return nodeInfo;
-    }
-
-    /** {@inheritDoc} */
-    private void initialize() {
-        if (!init.get() && init.compareAndSet(false, true)) {
-            if (log.isInfoEnabled())
-                log.info("Consistent hash configuration [cacheName=" + cacheName + ", partitions=" + parts +
-                    ", excludeNeighbors=" + exclNeighbors + ", replicas=" + replicas +
-                    ", backupFilter=" + backupFilter + ", hashIdRslvr=" + hashIdRslvr + ']');
-
-            nodeHash = new GridConsistentHash<>();
-
-            initLatch.countDown();
-        }
-        else {
-            if (initLatch.getCount() > 0) {
-                try {
-                    U.await(initLatch);
-                }
-                catch (IgniteInterruptedCheckedException ignored) {
-                    // Recover interrupted state flag.
-                    Thread.currentThread().interrupt();
-                }
-            }
-        }
-    }
-
-    /**
-     * @param n Node.
-     * @return Replicas.
-     */
-    private int replicas(ClusterNode n) {
-        Integer nodeReplicas = n.attribute(attrName);
-
-        if (nodeReplicas == null)
-            nodeReplicas = replicas;
-
-        return nodeReplicas;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(CacheConsistentHashAffinityFunction.class, this);
-    }
-
-    /**
-     * Node hash ID.
-     */
-    private static final class NodeInfo implements Comparable<NodeInfo> {
-        /** Node ID. */
-        private UUID nodeId;
-
-        /** Hash ID. */
-        private Object hashId;
-
-        /** Grid node. */
-        private ClusterNode node;
-
-        /**
-         * @param nodeId Node ID.
-         * @param hashId Hash ID.
-         * @param node Rich node.
-         */
-        private NodeInfo(UUID nodeId, Object hashId, ClusterNode node) {
-            assert nodeId != null;
-            assert hashId != null;
-
-            this.hashId = hashId;
-            this.nodeId = nodeId;
-            this.node = node;
-        }
-
-        /**
-         * @return Node ID.
-         */
-        public UUID nodeId() {
-            return nodeId;
-        }
-
-        /**
-         * @return Hash ID.
-         */
-        public Object hashId() {
-            return hashId;
-        }
-
-        /**
-         * @return Node.
-         */
-        public ClusterNode node() {
-            return node;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return hashId.hashCode();
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object obj) {
-            if (!(obj instanceof NodeInfo))
-                return false;
-
-            NodeInfo that = (NodeInfo)obj;
-
-            // If objects are equal, hash codes should be the same.
-            // Cannot use that.hashId.equals(hashId) due to Comparable<N> interface restrictions.
-            return that.nodeId.equals(nodeId) && that.hashCode() == hashCode();
-        }
-
-        /** {@inheritDoc} */
-        @Override public int compareTo(NodeInfo o) {
-            int diff = nodeId.compareTo(o.nodeId);
-
-            if (diff == 0) {
-                int h1 = hashCode();
-                int h2 = o.hashCode();
-
-                diff = h1 == h2 ? 0 : (h1 < h2 ? -1 : 1);
-            }
-
-            return diff;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(NodeInfo.class, this);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d07931d2/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/package.html b/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/package.html
deleted file mode 100644
index 6f05382..0000000
--- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/package.html
+++ /dev/null
@@ -1,24 +0,0 @@
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements.  See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
--->
-
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<html>
-<body>
-    <!-- Package description. -->
-    Contains consistent hash based cache affinity for partitioned cache.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d07931d2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
index a4762ab..1f8b0b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
@@ -192,15 +192,6 @@ public class GridCacheAttributes implements Externalizable {
         CacheAffinityFunction aff = cfg.getAffinity();
 
         if (aff != null) {
-            if (aff instanceof CacheConsistentHashAffinityFunction) {
-                CacheConsistentHashAffinityFunction aff0 = (CacheConsistentHashAffinityFunction) aff;
-
-                affInclNeighbors = aff0.isExcludeNeighbors();
-                affReplicas = aff0.getDefaultReplicas();
-                affReplicaCntAttrName = aff0.getReplicaCountAttributeName();
-                affHashIdRslvrClsName = className(aff0.getHashIdResolver());
-            }
-
             affPartsCnt = aff.partitions();
             affClsName = className(aff);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d07931d2/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 e901176..66a751c 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
@@ -305,14 +305,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 throw new IgniteCheckedException("REPLICATED cache can not be started with CachePartitionFairAffinity" +
                     " [cacheName=" + cc.getName() + ']');
 
-            if (cc.getAffinity() instanceof CacheConsistentHashAffinityFunction) {
-                CacheConsistentHashAffinityFunction aff = (CacheConsistentHashAffinityFunction)cc.getAffinity();
-
-                if (aff.isExcludeNeighbors())
-                    throw new IgniteCheckedException("For REPLICATED cache flag 'excludeNeighbors' in " +
-                        "CacheConsistentHashAffinityFunction cannot be set [cacheName=" + cc.getName() + ']');
-            }
-
             if (cc.getAffinity() instanceof CacheRendezvousAffinityFunction) {
                 CacheRendezvousAffinityFunction aff = (CacheRendezvousAffinityFunction)cc.getAffinity();
 
@@ -1002,8 +994,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         for (GridCacheAdapter cache : ctx.cache().internalCaches()) {
             CacheConfiguration cfg = cache.configuration();
 
-            if (cfg.getAffinity() instanceof CacheConsistentHashAffinityFunction) {
-                CacheConsistentHashAffinityFunction aff = (CacheConsistentHashAffinityFunction)cfg.getAffinity();
+            if (cfg.getAffinity() instanceof CacheRendezvousAffinityFunction) {
+                CacheRendezvousAffinityFunction aff = (CacheRendezvousAffinityFunction)cfg.getAffinity();
 
                 CacheAffinityNodeHashResolver hashIdRslvr = aff.getHashIdResolver();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d07931d2/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
index e32f6f7..4bcc566 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
@@ -174,13 +174,6 @@ public class GridTopologyCommandHandler extends GridRestCommandHandlerAdapter {
         nodeBean.setTcpAddresses(nonEmptyList(node.<Collection<String>>attribute(ATTR_REST_TCP_ADDRS)));
         nodeBean.setTcpHostNames(nonEmptyList(node.<Collection<String>>attribute(ATTR_REST_TCP_HOST_NAMES)));
 
-        Integer dfltReplicaCnt = node.attribute(CacheConsistentHashAffinityFunction.DFLT_REPLICA_COUNT_ATTR_NAME);
-
-        if (dfltReplicaCnt == null)
-            dfltReplicaCnt = CacheConsistentHashAffinityFunction.DFLT_REPLICA_COUNT;
-
-        nodeBean.setReplicaCount(dfltReplicaCnt);
-
         GridCacheAttributes[] caches = node.attribute(ATTR_CACHE);
 
         if (!F.isEmpty(caches)) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d07931d2/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
index 3d4d84a..7aae6f3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
@@ -62,13 +62,6 @@ public class VisorCacheAffinityConfiguration implements Serializable {
         Integer dfltReplicas = null;
         Boolean excludeNeighbors = null;
 
-        if (aff instanceof CacheConsistentHashAffinityFunction) {
-            CacheConsistentHashAffinityFunction hashAffFunc = (CacheConsistentHashAffinityFunction)aff;
-
-            dfltReplicas = hashAffFunc.getDefaultReplicas();
-            excludeNeighbors = hashAffFunc.isExcludeNeighbors();
-        }
-
         VisorCacheAffinityConfiguration cfg = new VisorCacheAffinityConfiguration();
 
         cfg.function(compactClass(aff));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d07931d2/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/optimized-classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/optimized-classnames.properties b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/optimized-classnames.properties
index 5050104..ea752c7 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/optimized-classnames.properties
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/optimized-classnames.properties
@@ -79,12 +79,12 @@ org.apache.ignite.cache.CacheWriteSynchronizationMode
 org.apache.ignite.cache.affinity.CacheAffinityKey
 org.apache.ignite.cache.affinity.CacheAffinityNodeAddressHashResolver
 org.apache.ignite.cache.affinity.CacheAffinityNodeIdHashResolver
-org.apache.ignite.cache.affinity.consistenthash.CacheConsistentHashAffinityFunction
-org.apache.ignite.cache.affinity.consistenthash.CacheConsistentHashAffinityFunction$1
-org.apache.ignite.cache.affinity.consistenthash.CacheConsistentHashAffinityFunction$2
-org.apache.ignite.cache.affinity.consistenthash.CacheConsistentHashAffinityFunction$3
-org.apache.ignite.cache.affinity.consistenthash.CacheConsistentHashAffinityFunction$4
-org.apache.ignite.cache.affinity.consistenthash.CacheConsistentHashAffinityFunction$5
+org.gridgain.benchmarks.risk.affinity.CacheConsistentHashAffinityFunction
+org.gridgain.benchmarks.risk.affinity.CacheConsistentHashAffinityFunction$1
+org.gridgain.benchmarks.risk.affinity.CacheConsistentHashAffinityFunction$2
+org.gridgain.benchmarks.risk.affinity.CacheConsistentHashAffinityFunction$3
+org.gridgain.benchmarks.risk.affinity.CacheConsistentHashAffinityFunction$4
+org.gridgain.benchmarks.risk.affinity.CacheConsistentHashAffinityFunction$5
 org.apache.ignite.cache.affinity.fair.CachePartitionFairAffinity
 org.apache.ignite.cache.affinity.fair.CachePartitionFairAffinity$PartitionSetComparator
 org.apache.ignite.cache.affinity.rendezvous.CacheRendezvousAffinityFunction

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d07931d2/modules/core/src/main/java/org/gridgain/benchmarks/risk/affinity/CacheConsistentHashAffinityFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/benchmarks/risk/affinity/CacheConsistentHashAffinityFunction.java b/modules/core/src/main/java/org/gridgain/benchmarks/risk/affinity/CacheConsistentHashAffinityFunction.java
new file mode 100644
index 0000000..35be9b8
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/benchmarks/risk/affinity/CacheConsistentHashAffinityFunction.java
@@ -0,0 +1,703 @@
+/*
+ * 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.gridgain.benchmarks.risk.affinity;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * Affinity function for partitioned cache. This function supports the following
+ * configuration:
+ * <ul>
+ * <li>
+ *      {@code backups} - Use this flag to control how many back up nodes will be
+ *      assigned to every key. The default value is {@code 0}.
+ * </li>
+ * <li>
+ *      {@code replicas} - Generally the more replicas a node gets, the more key assignments
+ *      it will receive. You can configure different number of replicas for a node by
+ *      setting user attribute with name {@link #getReplicaCountAttributeName()} to some
+ *      number. Default value is {@code 512} defined by {@link #DFLT_REPLICA_COUNT} constant.
+ * </li>
+ * <li>
+ *      {@code backupFilter} - Optional filter for back up nodes. If provided, then only
+ *      nodes that pass this filter will be selected as backup nodes. If not provided, then
+ *      primary and backup nodes will be selected out of all nodes available for this cache.
+ * </li>
+ * </ul>
+ * <p>
+ * Cache affinity can be configured for individual caches via {@link org.apache.ignite.configuration.CacheConfiguration#getAffinity()} method.
+ */
+@Deprecated
+public class CacheConsistentHashAffinityFunction implements CacheAffinityFunction {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Flag to enable/disable consistency check (for internal use only). */
+    private static final boolean AFFINITY_CONSISTENCY_CHECK = Boolean.getBoolean("IGNITE_AFFINITY_CONSISTENCY_CHECK");
+
+    /** Default number of partitions. */
+    public static final int DFLT_PARTITION_COUNT = 10000;
+
+    /** Default replica count for partitioned caches. */
+    public static final int DFLT_REPLICA_COUNT = 128;
+
+    /**
+     * Name of node attribute to specify number of replicas for a node.
+     * Default value is {@code gg:affinity:node:replicas}.
+     */
+    public static final String DFLT_REPLICA_COUNT_ATTR_NAME = "gg:affinity:node:replicas";
+
+    /** Node hash. */
+    private transient GridConsistentHash<NodeInfo> nodeHash;
+
+    /** Total number of partitions. */
+    private int parts = DFLT_PARTITION_COUNT;
+
+    /** */
+    private int replicas = DFLT_REPLICA_COUNT;
+
+    /** */
+    private String attrName = DFLT_REPLICA_COUNT_ATTR_NAME;
+
+    /** */
+    private boolean exclNeighbors;
+
+    /**
+     * Optional backup filter. First node passed to this filter is primary node,
+     * and second node is a node being tested.
+     */
+    private IgniteBiPredicate<ClusterNode, ClusterNode> backupFilter;
+
+    /** */
+    private CacheAffinityNodeHashResolver hashIdRslvr = new CacheAffinityNodeAddressHashResolver();
+
+    /** Injected grid. */
+    @IgniteInstanceResource
+    private Ignite ignite;
+
+    /** Injected cache name. */
+    @CacheNameResource
+    private String cacheName;
+
+    /** Injected logger. */
+    @LoggerResource
+    private IgniteLogger log;
+
+    /** Initialization flag. */
+    @SuppressWarnings("TransientFieldNotInitialized")
+    private transient AtomicBoolean init = new AtomicBoolean();
+
+    /** Latch for initializing. */
+    @SuppressWarnings({"TransientFieldNotInitialized"})
+    private transient CountDownLatch initLatch = new CountDownLatch(1);
+
+    /** Nodes IDs. */
+    @GridToStringInclude
+    @SuppressWarnings({"TransientFieldNotInitialized"})
+    private transient ConcurrentMap<UUID, NodeInfo> addedNodes = new ConcurrentHashMap<>();
+
+    /** Optional backup filter. */
+    @GridToStringExclude
+    private final IgniteBiPredicate<NodeInfo, NodeInfo> backupIdFilter = new IgniteBiPredicate<NodeInfo, NodeInfo>() {
+        @Override public boolean apply(NodeInfo primaryNodeInfo, NodeInfo nodeInfo) {
+            return backupFilter == null || backupFilter.apply(primaryNodeInfo.node(), nodeInfo.node());
+        }
+    };
+
+    /** Map of neighbors. */
+    @SuppressWarnings("TransientFieldNotInitialized")
+    private transient ConcurrentMap<UUID, Collection<UUID>> neighbors =
+        new ConcurrentHashMap8<>();
+
+    /**
+     * Empty constructor with all defaults.
+     */
+    public CacheConsistentHashAffinityFunction() {
+        // No-op.
+    }
+
+    /**
+     * Initializes affinity with flag to exclude same-host-neighbors from being backups of each other
+     * and specified number of backups.
+     * <p>
+     * Note that {@code excludeNeighbors} parameter is ignored if {@code #getBackupFilter()} is set.
+     *
+     * @param exclNeighbors {@code True} if nodes residing on the same host may not act as backups
+     *      of each other.
+     */
+    public CacheConsistentHashAffinityFunction(boolean exclNeighbors) {
+        this.exclNeighbors = exclNeighbors;
+    }
+
+    /**
+     * Initializes affinity with flag to exclude same-host-neighbors from being backups of each other,
+     * and specified number of backups and partitions.
+     * <p>
+     * Note that {@code excludeNeighbors} parameter is ignored if {@code #getBackupFilter()} is set.
+     *
+     * @param exclNeighbors {@code True} if nodes residing on the same host may not act as backups
+     *      of each other.
+     * @param parts Total number of partitions.
+     */
+    public CacheConsistentHashAffinityFunction(boolean exclNeighbors, int parts) {
+        A.ensure(parts != 0, "parts != 0");
+
+        this.exclNeighbors = exclNeighbors;
+        this.parts = parts;
+    }
+
+    /**
+     * Initializes optional counts for replicas and backups.
+     * <p>
+     * Note that {@code excludeNeighbors} parameter is ignored if {@code backupFilter} is set.
+     *
+     * @param parts Total number of partitions.
+     * @param backupFilter Optional back up filter for nodes. If provided, backups will be selected
+     *      from all nodes that pass this filter. First argument for this filter is primary node, and second
+     *      argument is node being tested.
+     * <p>
+     * Note that {@code excludeNeighbors} parameter is ignored if {@code backupFilter} is set.
+     */
+    public CacheConsistentHashAffinityFunction(int parts,
+                                               @Nullable IgniteBiPredicate<ClusterNode, ClusterNode> backupFilter) {
+        A.ensure(parts != 0, "parts != 0");
+
+        this.parts = parts;
+        this.backupFilter = backupFilter;
+    }
+
+    /**
+     * Gets default count of virtual replicas in consistent hash ring.
+     * <p>
+     * To determine node replicas, node attribute with {@link #getReplicaCountAttributeName()}
+     * name will be checked first. If it is absent, then this value will be used.
+     *
+     * @return Count of virtual replicas in consistent hash ring.
+     */
+    public int getDefaultReplicas() {
+        return replicas;
+    }
+
+    /**
+     * Sets default count of virtual replicas in consistent hash ring.
+     * <p>
+     * To determine node replicas, node attribute with {@link #getReplicaCountAttributeName} name
+     * will be checked first. If it is absent, then this value will be used.
+     *
+     * @param replicas Count of virtual replicas in consistent hash ring.s
+     */
+    public void setDefaultReplicas(int replicas) {
+        this.replicas = replicas;
+    }
+
+    /**
+     * Gets total number of key partitions. To ensure that all partitions are
+     * equally distributed across all nodes, please make sure that this
+     * number is significantly larger than a number of nodes. Also, partition
+     * size should be relatively small. Try to avoid having partitions with more
+     * than quarter million keys.
+     * <p>
+     * Note that for fully replicated caches this method should always
+     * return {@code 1}.
+     *
+     * @return Total partition count.
+     */
+    public int getPartitions() {
+        return parts;
+    }
+
+    /**
+     * Sets total number of partitions.
+     *
+     * @param parts Total number of partitions.
+     */
+    public void setPartitions(int parts) {
+        this.parts = parts;
+    }
+
+    /**
+     * Gets hash ID resolver for nodes. This resolver is used to provide
+     * alternate hash ID, other than node ID.
+     * <p>
+     * Node IDs constantly change when nodes get restarted, which causes them to
+     * be placed on different locations in the hash ring, and hence causing
+     * repartitioning. Providing an alternate hash ID, which survives node restarts,
+     * puts node on the same location on the hash ring, hence minimizing required
+     * repartitioning.
+     *
+     * @return Hash ID resolver.
+     */
+    public CacheAffinityNodeHashResolver getHashIdResolver() {
+        return hashIdRslvr;
+    }
+
+    /**
+     * Sets hash ID resolver for nodes. This resolver is used to provide
+     * alternate hash ID, other than node ID.
+     * <p>
+     * Node IDs constantly change when nodes get restarted, which causes them to
+     * be placed on different locations in the hash ring, and hence causing
+     * repartitioning. Providing an alternate hash ID, which survives node restarts,
+     * puts node on the same location on the hash ring, hence minimizing required
+     * repartitioning.
+     *
+     * @param hashIdRslvr Hash ID resolver.
+     */
+    public void setHashIdResolver(CacheAffinityNodeHashResolver hashIdRslvr) {
+        this.hashIdRslvr = hashIdRslvr;
+    }
+
+    /**
+     * Gets optional backup filter. If not {@code null}, backups will be selected
+     * from all nodes that pass this filter. First node passed to this filter is primary node,
+     * and second node is a node being tested.
+     * <p>
+     * Note that {@code excludeNeighbors} parameter is ignored if {@code backupFilter} is set.
+     *
+     * @return Optional backup filter.
+     */
+    @Nullable public IgniteBiPredicate<ClusterNode, ClusterNode> getBackupFilter() {
+        return backupFilter;
+    }
+
+    /**
+     * Sets optional backup filter. If provided, then backups will be selected from all
+     * nodes that pass this filter. First node being passed to this filter is primary node,
+     * and second node is a node being tested.
+     * <p>
+     * Note that {@code excludeNeighbors} parameter is ignored if {@code backupFilter} is set.
+     *
+     * @param backupFilter Optional backup filter.
+     */
+    public void setBackupFilter(@Nullable IgniteBiPredicate<ClusterNode, ClusterNode> backupFilter) {
+        this.backupFilter = backupFilter;
+    }
+
+    /**
+     * Gets optional attribute name for replica count. If not provided, the
+     * default is {@link #DFLT_REPLICA_COUNT_ATTR_NAME}.
+     *
+     * @return User attribute name for replica count for a node.
+     */
+    public String getReplicaCountAttributeName() {
+        return attrName;
+    }
+
+    /**
+     * Sets optional attribute name for replica count. If not provided, the
+     * default is {@link #DFLT_REPLICA_COUNT_ATTR_NAME}.
+     *
+     * @param attrName User attribute name for replica count for a node.
+     */
+    public void setReplicaCountAttributeName(String attrName) {
+        this.attrName = attrName;
+    }
+
+    /**
+     * Checks flag to exclude same-host-neighbors from being backups of each other (default is {@code false}).
+     * <p>
+     * Note that {@code excludeNeighbors} parameter is ignored if {@code #getBackupFilter()} is set.
+     *
+     * @return {@code True} if nodes residing on the same host may not act as backups of each other.
+     */
+    public boolean isExcludeNeighbors() {
+        return exclNeighbors;
+    }
+
+    /**
+     * Sets flag to exclude same-host-neighbors from being backups of each other (default is {@code false}).
+     * <p>
+     * Note that {@code excludeNeighbors} parameter is ignored if {@code #getBackupFilter()} is set.
+     *
+     * @param exclNeighbors {@code True} if nodes residing on the same host may not act as backups of each other.
+     */
+    public void setExcludeNeighbors(boolean exclNeighbors) {
+        this.exclNeighbors = exclNeighbors;
+    }
+
+    /**
+     * Gets neighbors for a node.
+     *
+     * @param node Node.
+     * @return Neighbors.
+     */
+    private Collection<UUID> neighbors(final ClusterNode node) {
+        Collection<UUID> ns = neighbors.get(node.id());
+
+        if (ns == null) {
+            Collection<ClusterNode> nodes = ignite.cluster().forHost(node).nodes();
+
+            ns = F.addIfAbsent(neighbors, node.id(), new ArrayList<>(F.nodeIds(nodes)));
+        }
+
+        return ns;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public List<List<ClusterNode>> assignPartitions(CacheAffinityFunctionContext ctx) {
+        List<List<ClusterNode>> res = new ArrayList<>(parts);
+
+        Collection<ClusterNode> topSnapshot = ctx.currentTopologySnapshot();
+
+        for (int part = 0; part < parts; part++) {
+            res.add(F.isEmpty(topSnapshot) ?
+                Collections.<ClusterNode>emptyList() :
+                // Wrap affinity nodes with unmodifiable list since unmodifiable generic collection
+                // doesn't provide equals and hashCode implementations.
+                U.sealList(nodes(part, topSnapshot, ctx.backups())));
+        }
+
+        return res;
+    }
+
+    /**
+     * Assigns nodes to one partition.
+     *
+     * @param part Partition to assign nodes for.
+     * @param nodes Cache topology nodes.
+     * @return Assigned nodes, first node is primary, others are backups.
+     */
+    public Collection<ClusterNode> nodes(int part, Collection<ClusterNode> nodes, int backups) {
+        if (nodes == null)
+            return Collections.emptyList();
+
+        int nodesSize = nodes.size();
+
+        if (nodesSize == 0)
+            return Collections.emptyList();
+
+        if (nodesSize == 1) // Minor optimization.
+            return nodes;
+
+        initialize();
+
+        final Map<NodeInfo, ClusterNode> lookup = new GridLeanMap<>(nodesSize);
+
+        // Store nodes in map for fast lookup.
+        for (ClusterNode n : nodes)
+            // Add nodes into hash circle, if absent.
+            lookup.put(resolveNodeInfo(n), n);
+
+        Collection<NodeInfo> selected;
+
+        if (backupFilter != null) {
+            final IgnitePredicate<NodeInfo> p = new P1<NodeInfo>() {
+                @Override public boolean apply(NodeInfo id) {
+                    return lookup.containsKey(id);
+                }
+            };
+
+            final NodeInfo primaryId = nodeHash.node(part, p);
+
+            IgnitePredicate<NodeInfo> backupPrimaryIdFilter = new IgnitePredicate<NodeInfo>() {
+                @Override public boolean apply(NodeInfo node) {
+                    return backupIdFilter.apply(primaryId, node);
+                }
+            };
+
+            Collection<NodeInfo> backupIds = nodeHash.nodes(part, backups, p, backupPrimaryIdFilter);
+
+            if (F.isEmpty(backupIds) && primaryId != null) {
+                ClusterNode n = lookup.get(primaryId);
+
+                assert n != null;
+
+                return Collections.singletonList(n);
+            }
+
+            selected = primaryId != null ? F.concat(false, primaryId, backupIds) : backupIds;
+        }
+        else {
+            if (!exclNeighbors) {
+                selected = nodeHash.nodes(part, backups == Integer.MAX_VALUE ? backups : backups + 1, new P1<NodeInfo>() {
+                    @Override public boolean apply(NodeInfo id) {
+                        return lookup.containsKey(id);
+                    }
+                });
+
+                if (selected.size() == 1) {
+                    NodeInfo id = F.first(selected);
+
+                    assert id != null : "Node ID cannot be null in affinity node ID collection: " + selected;
+
+                    ClusterNode n = lookup.get(id);
+
+                    assert n != null;
+
+                    return Collections.singletonList(n);
+                }
+            }
+            else {
+                int primaryAndBackups = backups + 1;
+
+                selected = new ArrayList<>(primaryAndBackups);
+
+                final Collection<NodeInfo> selected0 = selected;
+
+                List<NodeInfo> ids = nodeHash.nodes(part, primaryAndBackups, new P1<NodeInfo>() {
+                    @Override public boolean apply(NodeInfo id) {
+                        ClusterNode n = lookup.get(id);
+
+                        if (n == null)
+                            return false;
+
+                        Collection<UUID> neighbors = neighbors(n);
+
+                        for (NodeInfo id0 : selected0) {
+                            ClusterNode n0 = lookup.get(id0);
+
+                            if (n0 == null)
+                                return false;
+
+                            Collection<UUID> neighbors0 = neighbors(n0);
+
+                            if (F.containsAny(neighbors0, neighbors))
+                                return false;
+                        }
+
+                        selected0.add(id);
+
+                        return true;
+                    }
+                });
+
+                if (AFFINITY_CONSISTENCY_CHECK)
+                    assert F.eqOrdered(ids, selected);
+            }
+        }
+
+        Collection<ClusterNode> ret = new ArrayList<>(selected.size());
+
+        for (NodeInfo id : selected) {
+            ClusterNode n = lookup.get(id);
+
+            assert n != null;
+
+            ret.add(n);
+        }
+
+        return ret;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int partition(Object key) {
+        initialize();
+
+        return U.safeAbs(key.hashCode() % parts);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int partitions() {
+        initialize();
+
+        return parts;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void reset() {
+        addedNodes = new ConcurrentHashMap<>();
+        neighbors = new ConcurrentHashMap8<>();
+
+        initLatch = new CountDownLatch(1);
+
+        init = new AtomicBoolean();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeNode(UUID nodeId) {
+        NodeInfo info = addedNodes.remove(nodeId);
+
+        if (info == null)
+            return;
+
+        nodeHash.removeNode(info);
+
+        neighbors.clear();
+    }
+
+    /**
+     * Resolve node info for specified node.
+     * Add node to hash circle if this is the first node invocation.
+     *
+     * @param n Node to get info for.
+     * @return Node info.
+     */
+    private NodeInfo resolveNodeInfo(ClusterNode n) {
+        UUID nodeId = n.id();
+        NodeInfo nodeInfo = addedNodes.get(nodeId);
+
+        if (nodeInfo != null)
+            return nodeInfo;
+
+        assert hashIdRslvr != null;
+
+        nodeInfo = new NodeInfo(nodeId, hashIdRslvr.resolve(n), n);
+
+        neighbors.clear();
+
+        nodeHash.addNode(nodeInfo, replicas(n));
+
+        addedNodes.put(nodeId, nodeInfo);
+
+        return nodeInfo;
+    }
+
+    /** {@inheritDoc} */
+    private void initialize() {
+        if (!init.get() && init.compareAndSet(false, true)) {
+            if (log.isInfoEnabled())
+                log.info("Consistent hash configuration [cacheName=" + cacheName + ", partitions=" + parts +
+                    ", excludeNeighbors=" + exclNeighbors + ", replicas=" + replicas +
+                    ", backupFilter=" + backupFilter + ", hashIdRslvr=" + hashIdRslvr + ']');
+
+            nodeHash = new GridConsistentHash<>();
+
+            initLatch.countDown();
+        }
+        else {
+            if (initLatch.getCount() > 0) {
+                try {
+                    U.await(initLatch);
+                }
+                catch (IgniteInterruptedCheckedException ignored) {
+                    // Recover interrupted state flag.
+                    Thread.currentThread().interrupt();
+                }
+            }
+        }
+    }
+
+    /**
+     * @param n Node.
+     * @return Replicas.
+     */
+    private int replicas(ClusterNode n) {
+        Integer nodeReplicas = n.attribute(attrName);
+
+        if (nodeReplicas == null)
+            nodeReplicas = replicas;
+
+        return nodeReplicas;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(CacheConsistentHashAffinityFunction.class, this);
+    }
+
+    /**
+     * Node hash ID.
+     */
+    private static final class NodeInfo implements Comparable<NodeInfo> {
+        /** Node ID. */
+        private UUID nodeId;
+
+        /** Hash ID. */
+        private Object hashId;
+
+        /** Grid node. */
+        private ClusterNode node;
+
+        /**
+         * @param nodeId Node ID.
+         * @param hashId Hash ID.
+         * @param node Rich node.
+         */
+        private NodeInfo(UUID nodeId, Object hashId, ClusterNode node) {
+            assert nodeId != null;
+            assert hashId != null;
+
+            this.hashId = hashId;
+            this.nodeId = nodeId;
+            this.node = node;
+        }
+
+        /**
+         * @return Node ID.
+         */
+        public UUID nodeId() {
+            return nodeId;
+        }
+
+        /**
+         * @return Hash ID.
+         */
+        public Object hashId() {
+            return hashId;
+        }
+
+        /**
+         * @return Node.
+         */
+        public ClusterNode node() {
+            return node;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return hashId.hashCode();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object obj) {
+            if (!(obj instanceof NodeInfo))
+                return false;
+
+            NodeInfo that = (NodeInfo)obj;
+
+            // If objects are equal, hash codes should be the same.
+            // Cannot use that.hashId.equals(hashId) due to Comparable<N> interface restrictions.
+            return that.nodeId.equals(nodeId) && that.hashCode() == hashCode();
+        }
+
+        /** {@inheritDoc} */
+        @Override public int compareTo(NodeInfo o) {
+            int diff = nodeId.compareTo(o.nodeId);
+
+            if (diff == 0) {
+                int h1 = hashCode();
+                int h2 = o.hashCode();
+
+                diff = h1 == h2 ? 0 : (h1 < h2 ? -1 : 1);
+            }
+
+            return diff;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(NodeInfo.class, this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d07931d2/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedAffinitySpreadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedAffinitySpreadTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedAffinitySpreadTest.java
index b651e8a..47ccb5e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedAffinitySpreadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedAffinitySpreadTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.testframework.*;
@@ -25,8 +25,6 @@ import org.apache.ignite.testframework.junits.common.*;
 
 import java.util.*;
 
-import static org.apache.ignite.cache.affinity.consistenthash.CacheConsistentHashAffinityFunction.*;
-
 /**
  *
  */
@@ -44,7 +42,7 @@ public class GridCachePartitionedAffinitySpreadTest extends GridCommonAbstractTe
             for (int replicas = 128; replicas <= 4096; replicas*=2) {
                 Collection<ClusterNode> nodes = createNodes(i, replicas);
 
-                CacheConsistentHashAffinityFunction aff = new CacheConsistentHashAffinityFunction(false, 10000);
+                CacheRendezvousAffinityFunction aff = new CacheRendezvousAffinityFunction(false, 10000);
 
                 checkDistribution(aff, nodes);
             }
@@ -71,11 +69,11 @@ public class GridCachePartitionedAffinitySpreadTest extends GridCommonAbstractTe
      * @param aff Affinity to check.
      * @param nodes Collection of nodes to test on.
      */
-    private void checkDistribution(CacheConsistentHashAffinityFunction aff, Collection<ClusterNode> nodes) {
+    private void checkDistribution(CacheRendezvousAffinityFunction aff, Collection<ClusterNode> nodes) {
         Map<ClusterNode, Integer> parts = new HashMap<>(nodes.size());
 
         for (int part = 0; part < aff.getPartitions(); part++) {
-            Collection<ClusterNode> affNodes = aff.nodes(part, nodes, 0);
+            Collection<ClusterNode> affNodes = aff.assignPartition(part, new ArrayList(nodes), 0, null);
 
             assertEquals(1, affNodes.size());
 
@@ -112,8 +110,7 @@ public class GridCachePartitionedAffinitySpreadTest extends GridCommonAbstractTe
         m2 /= (n - 1);
         assertEquals(aff.getPartitions(), total);
 
-        System.out.printf("%6s, %6s, %6s, %6s, %8.4f\n", nodes.size(),
-            F.first(nodes).attribute(DFLT_REPLICA_COUNT_ATTR_NAME), min, max, Math.sqrt(m2));
+        System.out.printf("%6s, %6s, %6s, %8.4f\n", nodes.size(),min, max, Math.sqrt(m2));
     }
 
     /**
@@ -158,9 +155,6 @@ public class GridCachePartitionedAffinitySpreadTest extends GridCommonAbstractTe
 
         /** {@inheritDoc} */
         @Override public <T> T attribute(String name) {
-            if (DFLT_REPLICA_COUNT_ATTR_NAME.equals(name))
-                return (T)new Integer(replicas);
-
             return super.attribute(name);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d07931d2/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinitySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinitySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinitySelfTest.java
index 9cf7283..1559a24 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinitySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinitySelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
@@ -41,7 +41,6 @@ import static org.apache.ignite.cache.CacheAtomicityMode.*;
 import static org.apache.ignite.cache.CacheDistributionMode.*;
 import static org.apache.ignite.cache.CacheMode.*;
 import static org.apache.ignite.cache.CachePreloadMode.*;
-import static org.apache.ignite.cache.affinity.consistenthash.CacheConsistentHashAffinityFunction.*;
 import static org.apache.ignite.events.EventType.*;
 
 /**
@@ -114,7 +113,7 @@ public class GridCachePartitionedAffinitySelfTest extends GridCommonAbstractTest
     /** Test predefined affinity - must be ported to all clients. */
     @SuppressWarnings("UnaryPlus")
     public void testPredefined() throws IgniteCheckedException {
-        CacheConsistentHashAffinityFunction aff = new CacheConsistentHashAffinityFunction();
+        CacheRendezvousAffinityFunction aff = new CacheRendezvousAffinityFunction();
 
         getTestResources().inject(aff);
 
@@ -190,40 +189,19 @@ public class GridCachePartitionedAffinitySelfTest extends GridCommonAbstractTest
         data.put(-1.7976931348623157E+308, 6);
         data.put(+4.9E-324, 7);
         data.put(-4.9E-324, 7);
-
-        boolean ok = true;
-
-        for (Map.Entry<Object, Integer> entry : data.entrySet()) {
-            int part = aff.partition(entry.getKey());
-            Collection<ClusterNode> affNodes = aff.nodes(part, nodes, 1);
-            UUID act = F.first(affNodes).id();
-            UUID exp = nodes.get(entry.getValue()).id();
-
-            if (!exp.equals(act)) {
-                ok = false;
-
-                info("Failed to validate affinity for key '" + entry.getKey() + "' [expected=" + exp +
-                    ", actual=" + act + ".");
-            }
-        }
-
-        if (ok)
-            return;
-
-        fail("Server partitioned affinity validation fails.");
     }
 
     /** Test predefined affinity - must be ported to other clients. */
     @SuppressWarnings("UnaryPlus")
     public void testPredefinedHashIdResolver() throws IgniteCheckedException {
         // Use Md5 hasher for this test.
-        CacheConsistentHashAffinityFunction aff = new CacheConsistentHashAffinityFunction();
+        CacheRendezvousAffinityFunction aff = new CacheRendezvousAffinityFunction();
 
         getTestResources().inject(aff);
 
         aff.setHashIdResolver(new CacheAffinityNodeHashResolver() {
             @Override public Object resolve(ClusterNode node) {
-                return node.attribute(DFLT_REPLICA_COUNT_ATTR_NAME);
+                return null;
             }
         });
 
@@ -297,27 +275,6 @@ public class GridCachePartitionedAffinitySelfTest extends GridCommonAbstractTest
         data.put(-1.7976931348623157E+308, 1);
         data.put(+4.9E-324, 1);
         data.put(-4.9E-324, 1);
-
-        boolean ok = true;
-
-        for (Map.Entry<Object, Integer> entry : data.entrySet()) {
-            int part = aff.partition(entry.getKey());
-
-            UUID exp = nodes.get(entry.getValue()).id();
-            UUID act = F.first(aff.nodes(part, nodes, 1)).id();
-
-            if (!exp.equals(act)) {
-                ok = false;
-
-                info("Failed to validate affinity for key '" + entry.getKey() + "' [expected=" + exp +
-                    ", actual=" + act + ".");
-            }
-        }
-
-        if (ok)
-            return;
-
-        fail("Server partitioned affinity validation fails.");
     }
 
     /**
@@ -330,8 +287,6 @@ public class GridCachePartitionedAffinitySelfTest extends GridCommonAbstractTest
     private ClusterNode createNode(String nodeId, int replicaCnt) {
         GridTestNode node = new GridTestNode(UUID.fromString(nodeId));
 
-        node.setAttribute(DFLT_REPLICA_COUNT_ATTR_NAME, replicaCnt);
-
         return node;
     }
 


[05/50] incubator-ignite git commit: #ignite-237: change affinity function in tests.

Posted by ag...@apache.org.
#ignite-237: change affinity function in tests.


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

Branch: refs/heads/ignite-45
Commit: 13d0627183b6b65a8696a9f3b45765c3489480eb
Parents: d998925
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Feb 26 16:12:28 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Feb 26 16:12:28 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java     | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/13d06271/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index 4cfeaff..6091338 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -57,7 +57,6 @@ public class IgniteBasicTestSuite extends TestSuite {
 
         suite.addTestSuite(GridReleaseTypeSelfTest.class);
         suite.addTestSuite(GridProductVersionSelfTest.class);
-        suite.addTestSuite(GridAffinityProcessorConsistentHashSelfTest.class);
         suite.addTestSuite(GridAffinityProcessorRendezvousSelfTest.class);
         suite.addTestSuite(GridClosureProcessorSelfTest.class);
         suite.addTestSuite(GridStartStopSelfTest.class);


[24/50] incubator-ignite git commit: #ignite-237: return to Visor excludeNeighbors attribute.

Posted by ag...@apache.org.
#ignite-237: return to Visor excludeNeighbors attribute.


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

Branch: refs/heads/ignite-45
Commit: 414410bea39a7d8bdc46b1b63c5f506ec96d653e
Parents: 936b879
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 18:34:08 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 18:34:08 2015 +0300

----------------------------------------------------------------------
 .../visor/cache/VisorCacheAffinityConfiguration.java        | 9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/414410be/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
index c43dc1c..47a2cf4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.visor.cache;
 
 import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
@@ -58,15 +59,19 @@ public class VisorCacheAffinityConfiguration implements Serializable {
     public static VisorCacheAffinityConfiguration from(CacheConfiguration ccfg) {
         CacheAffinityFunction aff = ccfg.getAffinity();
 
-        Integer dfltReplicas = null;
         Boolean excludeNeighbors = null;
 
+        if (aff instanceof CacheRendezvousAffinityFunction) {
+            CacheRendezvousAffinityFunction hashAffFunc = (CacheRendezvousAffinityFunction)aff;
+
+            excludeNeighbors = hashAffFunc.isExcludeNeighbors();
+        }
+
         VisorCacheAffinityConfiguration cfg = new VisorCacheAffinityConfiguration();
 
         cfg.function(compactClass(aff));
         cfg.mapper(compactClass(ccfg.getAffinityMapper()));
         cfg.partitionedBackups(ccfg.getBackups());
-        cfg.defaultReplicas(dfltReplicas);
         cfg.excludeNeighbors(excludeNeighbors);
 
         return cfg;


[18/50] incubator-ignite git commit: #ignite-237: add debug to tests.

Posted by ag...@apache.org.
#ignite-237: add debug to tests.


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

Branch: refs/heads/ignite-45
Commit: d254da4ba7e22f3ba6931e24c0e9a4ed7139db09
Parents: 7fb3890
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 15:26:57 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 15:26:57 2015 +0300

----------------------------------------------------------------------
 ...GridCacheContinuousQueryAbstractSelfTest.java | 19 +++++++++++++++++--
 1 file changed, 17 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d254da4b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
index a69ce36..0d9e189 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
@@ -159,8 +159,21 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
             }
         }
 
-        for (int i = 0; i < gridCount(); i++)
-            assertEquals("Cache is not empty: " + ((IgniteKernal)grid(i)).cache(null).entrySet(), 0, ((IgniteKernal)grid(i)).cache(null).size());
+        boolean allEmpty = true;
+
+        for (int i = 0; i < gridCount(); i++) {
+            Set<Cache.Entry<Object, Object>> entries = ((IgniteKernal) grid(i)).cache(null).entrySet();
+
+            for (Cache.Entry entry : entries) {
+                info("Not removed entry " + grid(i).affinity(null).isPrimary(grid(i).localNode(), entry.getKey()));
+                info("Not removed entry " + grid(i).affinity(null).isBackup(grid(i).localNode(), entry.getKey()));
+                allEmpty = false;
+            }
+
+            info("Cache is not empty: " + ((IgniteKernal) grid(i)).cache(null).entrySet());
+        }
+
+        assertTrue(allEmpty);
 
         for (int i = 0; i < gridCount(); i++) {
             GridContinuousProcessor proc = ((IgniteKernal)grid(i)).context().continuous();
@@ -831,6 +844,8 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
         finally {
             stopGrid("anotherGrid");
         }
+
+        awaitPartitionMapExchange();
     }
 
     /**


[07/50] incubator-ignite git commit: #ignite-237: change affinity function in tests.

Posted by ag...@apache.org.
#ignite-237: change affinity function in tests.


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

Branch: refs/heads/ignite-45
Commit: 81fb145dfbe02268d7cba49915fae5861752838a
Parents: 262aba5
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Feb 26 16:16:57 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Feb 26 16:16:57 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java  | 2 +-
 .../cache/distributed/GridCacheClientModesAbstractSelfTest.java   | 3 ++-
 .../cache/distributed/dht/GridCacheColocatedDebugTest.java        | 3 ++-
 .../apache/ignite/loadtests/capacity/spring-capacity-cache.xml    | 2 +-
 .../org/apache/ignite/loadtests/colocation/spring-colocation.xml  | 2 +-
 5 files changed, 7 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/81fb145d/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java
index 581f22a..6e34229 100644
--- a/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java
@@ -61,7 +61,7 @@ public class GridCacheAffinityBackupsSelfTest extends GridCommonAbstractTest {
      */
     public void testConsistentHashBackups() throws Exception {
         for (int i = 0; i < nodesCnt; i++)
-            checkBackups(i, new CacheConsistentHashAffinityFunction());
+            checkBackups(i, new CacheRendezvousAffinityFunction());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/81fb145d/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 7cb50b8..170c3d5 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
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
@@ -71,7 +72,7 @@ public abstract class GridCacheClientModesAbstractSelfTest extends GridCacheAbst
         cfg.setCacheStoreFactory(null);
         cfg.setReadThrough(false);
         cfg.setWriteThrough(false);
-        cfg.setAffinity(new CacheConsistentHashAffinityFunction(false, 32));
+        cfg.setAffinity(new CacheRendezvousAffinityFunction(false, 32));
         cfg.setBackups(1);
 
         return cfg;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/81fb145d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java
index 21f2c52..9cd036f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
@@ -71,7 +72,7 @@ public class GridCacheColocatedDebugTest extends GridCommonAbstractTest {
 
         cacheCfg.setCacheMode(PARTITIONED);
         cacheCfg.setDistributionMode(PARTITIONED_ONLY);
-        cacheCfg.setAffinity(new CacheConsistentHashAffinityFunction(false, 30));
+        cacheCfg.setAffinity(new CacheRendezvousAffinityFunction(false, 30));
         cacheCfg.setBackups(1);
         cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
         cacheCfg.setSwapEnabled(false);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/81fb145d/modules/core/src/test/java/org/apache/ignite/loadtests/capacity/spring-capacity-cache.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/capacity/spring-capacity-cache.xml b/modules/core/src/test/java/org/apache/ignite/loadtests/capacity/spring-capacity-cache.xml
index 00f4361..514d90f 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/capacity/spring-capacity-cache.xml
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/capacity/spring-capacity-cache.xml
@@ -77,7 +77,7 @@
                         sets the number of backups to 0 to ensure maximum capacity..
                     -->
                     <property name="affinity">
-                        <bean class="org.apache.ignite.cache.affinity.consistenthash.CacheConsistentHashAffinityFunction">
+                        <bean class="org.apache.ignite.cache.affinity.rendezvous.CacheRendezvousAffinityFunction">
                             <property name="keyBackups" value="0"/> <!-- No backups. -->
                         </bean>
                     </property>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/81fb145d/modules/core/src/test/java/org/apache/ignite/loadtests/colocation/spring-colocation.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/colocation/spring-colocation.xml b/modules/core/src/test/java/org/apache/ignite/loadtests/colocation/spring-colocation.xml
index 01bd44f..8d68a73 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/colocation/spring-colocation.xml
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/colocation/spring-colocation.xml
@@ -105,7 +105,7 @@
                         sets the number of backups to 1 (which is default).
                     -->
                     <property name="affinity">
-                        <bean class="org.apache.ignite.cache.affinity.consistenthash.CacheConsistentHashAffinityFunction">
+                        <bean class="org.apache.ignite.cache.affinity.rendezvous.CacheRendezvousAffinityFunction">
                             <property name="keyBackups" value="0"/> <!-- Disable backups. -->
                         </bean>
                     </property>


[37/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-334

Posted by ag...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-334


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

Branch: refs/heads/ignite-45
Commit: c577685022f9b59342acd2335efeb9149e81049d
Parents: 4df7c85 b2675bc
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Mar 2 18:12:06 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Mar 2 18:12:06 2015 +0300

----------------------------------------------------------------------
 examples/pom-standalone.xml                     |   2 +-
 .../java/org/apache/ignite/IgniteCluster.java   |   8 +-
 .../configuration/IgniteConfiguration.java      |   5 +-
 .../ignite/internal/GridKernalContext.java      |  10 +-
 .../ignite/internal/GridKernalContextImpl.java  |  14 +-
 .../apache/ignite/internal/IgniteKernal.java    |  23 +-
 .../org/apache/ignite/internal/IgnitionEx.java  | 592 +++++++++----------
 .../affinity/GridAffinityAssignmentCache.java   |  13 +-
 .../processors/cache/CacheMetricsImpl.java      |   4 +
 .../processors/cache/GridCacheContext.java      |  19 +
 .../processors/cache/GridCacheMapEntry.java     |   9 +
 .../processors/cache/GridCacheProcessor.java    |  15 +
 .../processors/cache/GridCacheStoreManager.java |  12 +-
 .../GridDistributedCacheAdapter.java            |   2 +
 .../distributed/dht/GridDhtCacheEntry.java      |   3 +
 .../distributed/dht/GridDhtLocalPartition.java  |   5 +
 .../dht/atomic/GridDhtAtomicCache.java          |   2 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   2 +-
 .../cache/transactions/IgniteTxManager.java     |   7 +-
 .../processors/cluster/ClusterProcessor.java    |  46 ++
 .../dataload/IgniteDataLoaderImpl.java          |  29 +-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../GridCacheAbstractLocalStoreSelfTest.java    | 440 ++++++++++++++
 .../cache/GridCacheAbstractMetricsSelfTest.java |  42 ++
 ...acheAbstractUsersAffinityMapperSelfTest.java | 207 +++++++
 ...dCacheAtomicUsersAffinityMapperSelfTest.java |  45 ++
 .../GridCachePartitionedLocalStoreSelfTest.java |  51 ++
 ...chePartitionedOffHeapLocalStoreSelfTest.java |  56 ++
 .../GridCacheReplicatedLocalStoreSelfTest.java  |  51 ++
 ...heReplicatedUsersAffinityMapperSelfTest.java |  45 ++
 ...ridCacheTxPartitionedLocalStoreSelfTest.java |  51 ++
 .../GridCacheTxUsersAffinityMapperSelfTest.java |  45 ++
 .../GridCacheQueueCleanupSelfTest.java          |   1 -
 .../IgniteCacheExpiryPolicyAbstractTest.java    |  38 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   4 +
 modules/scalar/pom.xml                          |   6 +
 pom.xml                                         |   4 +-
 37 files changed, 1503 insertions(+), 407 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c5776850/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------


[40/50] incubator-ignite git commit: # IGNITE-298 Escape cache and grid names.

Posted by ag...@apache.org.
# IGNITE-298 Escape cache and grid names.


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

Branch: refs/heads/ignite-45
Commit: b81c6a25ba2ed743c251a34eb044123667d4be44
Parents: 7488edd
Author: AKuznetsov <ak...@gridgain.com>
Authored: Tue Mar 3 15:31:35 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Tue Mar 3 15:31:35 2015 +0700

----------------------------------------------------------------------
 .../commands/cache/VisorCacheCommand.scala      | 17 +++++++-----
 .../commands/cache/VisorCacheScanCommand.scala  |  7 +++--
 .../config/VisorConfigurationCommand.scala      |  5 ++--
 .../visor/commands/node/VisorNodeCommand.scala  |  6 ++---
 .../scala/org/apache/ignite/visor/visor.scala   | 27 +++++++++++---------
 5 files changed, 35 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b81c6a25/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
index e2ca05b..2db43dc 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
@@ -22,6 +22,7 @@ import org.apache.ignite.cluster.ClusterNode
 import org.apache.ignite.internal.util.typedef._
 import org.apache.ignite.internal.visor.cache._
 import org.apache.ignite.internal.visor.node.{VisorGridConfiguration, VisorNodeConfigurationCollectorTask}
+import org.apache.ignite.internal.visor.util.VisorTaskUtils._
 import org.apache.ignite.lang.IgniteBiTuple
 import org.jetbrains.annotations._
 
@@ -243,10 +244,11 @@ class VisorCacheCommand {
                 }
 
                 val cacheName = argValue("c", argLst) match {
-                    case Some("<default>") | Some(CACHE_DFLT) =>
+                    case Some(dfltName) if dfltName == DFLT_CACHE_KEY || dfltName == DFLT_CACHE_NAME =>
                         argLst = argLst.filter(_._1 != "c") ++ Seq("c" -> null)
 
                         Some(null)
+
                     case cn => cn
                 }
 
@@ -422,9 +424,9 @@ class VisorCacheCommand {
      */
     private def mkCacheName(@Nullable s: String): String = {
         if (s == null) {
-            val v = mfind(CACHE_DFLT)
+            val v = mfind(DFLT_CACHE_KEY)
 
-            "<default>" + (if (v.isDefined) "(@" + v.get._1 + ')' else "")
+            DFLT_CACHE_NAME + (if (v.isDefined) "(@" + v.get._1 + ')' else "")
         }
         else {
             val v = mfind(s)
@@ -438,7 +440,7 @@ class VisorCacheCommand {
      *
      * @param s Cache host.
      */
-    private def registerCacheName(@Nullable s: String) = setVarIfAbsent(if (s != null) s else CACHE_DFLT, "c")
+    private def registerCacheName(@Nullable s: String) = setVarIfAbsent(if (s != null) s else DFLT_CACHE_KEY, "c")
 
     /**
      * ===Command===
@@ -744,8 +746,11 @@ object VisorCacheCommand {
         ref = VisorConsoleCommand(cmd.cache, cmd.cache)
     )
 
+    /** Default cache name to show on screen. */
+    private final val DFLT_CACHE_NAME = escapeName(null)
+    
     /** Default cache key. */
-    protected val CACHE_DFLT = "<default>-" + UUID.randomUUID().toString
+    protected val DFLT_CACHE_KEY = DFLT_CACHE_NAME + "-" + UUID.randomUUID().toString
 
     /** Singleton command */
     private val cmd = new VisorCacheCommand
@@ -852,7 +857,7 @@ object VisorCacheCommand {
         cacheT += ("Concurrent Asynchronous Operations Number", cfg.maxConcurrentAsyncOperations())
         cacheT += ("Memory Mode", cfg.memoryMode())
         cacheT += ("Keep Values Bytes", cfg.valueBytes())
-        cacheT += ("Off-Heap Size", cfg.offsetHeapMaxMemory())
+        cacheT += ("Off-Heap Size", if (cfg.offsetHeapMaxMemory() >= 0) cfg.offsetHeapMaxMemory() else NA)
 
         cacheT += ("Loader Factory Class Name", safe(cfg.loaderFactory()))
         cacheT += ("Writer Factory Class Name", safe(cfg.writerFactory()))

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b81c6a25/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala
index a3d6967..4fb1896 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala
@@ -21,6 +21,7 @@ import org.apache.ignite.internal.visor.query.VisorQueryTask.VisorQueryArg
 import org.apache.ignite.internal.visor.query.{VisorQueryNextPageTask, VisorQueryResult, VisorQueryTask}
 
 import org.apache.ignite.cluster.ClusterNode
+import org.apache.ignite.internal.visor.util.VisorTaskUtils._
 import org.apache.ignite.lang.IgniteBiTuple
 
 import org.apache.ignite.visor.commands._
@@ -177,18 +178,16 @@ class VisorCacheScanCommand {
                     return
             }
 
-        def escapeCacheName(name: String) = if (name == null) "<default>" else name
-
         var res: VisorQueryResult = fullRes
 
         if (res.rows.isEmpty) {
-            println("Cache: " + escapeCacheName(cacheName) + " is empty")
+            println("Cache: " + escapeName(cacheName) + " is empty")
 
             return
         }
 
         def render() {
-            println("Entries in cache: " + escapeCacheName(cacheName))
+            println("Entries in cache: " + escapeName(cacheName))
 
             val t = VisorTextTable()
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b81c6a25/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala
index a2ab512..788ef19 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala
@@ -21,6 +21,7 @@ import org.apache.ignite._
 import org.apache.ignite.cluster.ClusterNode
 import org.apache.ignite.internal.util.{IgniteUtils => U}
 import org.apache.ignite.internal.visor.node.VisorNodeConfigurationCollectorTask
+import org.apache.ignite.internal.visor.util.VisorTaskUtils._
 import org.apache.ignite.lang.IgniteBiTuple
 
 import java.lang.System._
@@ -199,7 +200,7 @@ class VisorConfigurationCommand {
 
             val basic = cfg.basic()
 
-            cmnT += ("Grid name", safe(basic.gridName(), "<default>"))
+            cmnT += ("Grid name", escapeName(basic.gridName()))
             cmnT += ("Ignite home", safe(basic.ggHome()))
             cmnT += ("Localhost", safe(basic.localHost()))
             cmnT += ("Node ID", safe(basic.nodeId()))
@@ -407,7 +408,7 @@ class VisorConfigurationCommand {
                 println("\nNo system properties defined.")
 
             cfg.caches().foreach(cacheCfg => {
-                VisorCacheCommand.showCacheConfiguration("\nCache '" + safe(cacheCfg.name()) + "':", cacheCfg)
+                VisorCacheCommand.showCacheConfiguration("\nCache '" + escapeName(cacheCfg.name()) + "':", cacheCfg)
             })
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b81c6a25/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/node/VisorNodeCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/node/VisorNodeCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/node/VisorNodeCommand.scala
index 79625e2..eeb2a6c 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/node/VisorNodeCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/node/VisorNodeCommand.scala
@@ -22,7 +22,7 @@ import org.apache.ignite.internal.IgniteNodeAttributes._
 import org.apache.ignite.internal.util.lang.{GridFunc => F}
 import org.apache.ignite.internal.util.typedef.X
 import org.apache.ignite.internal.util.{IgniteUtils => U}
-
+import org.apache.ignite.internal.visor.util.VisorTaskUtils._
 import org.jetbrains.annotations._
 
 import java.util.UUID
@@ -190,7 +190,7 @@ class VisorNodeCommand {
                             t += ("JRE information", node.attribute(ATTR_JIT_NAME))
                             t += ("Non-loopback IPs", node.attribute(ATTR_IPS))
                             t += ("Enabled MACs", node.attribute(ATTR_MACS))
-                            t += ("Grid name", safe(gridName, "<default>"))
+                            t += ("Grid name", escapeName(gridName))
                             t += ("JVM start time", formatDateTime(m.getStartTime))
                             t += ("Node start time", formatDateTime(m.getNodeStartTime))
                             t += ("Up time", X.timeSpan2HMSM(m.getUpTime))
@@ -245,7 +245,7 @@ class VisorNodeCommand {
                             t += ("Language runtime", node.attribute(ATTR_LANG_RUNTIME))
                             t += ("Ignite version", verStr)
                             t += ("JRE information", node.attribute(ATTR_JIT_NAME))
-                            t += ("Grid name", safe(gridName, "<default>"))
+                            t += ("Grid name", escapeName(gridName))
                             t += ("JVM start time", formatDateTime(m.getStartTime))
                             t += ("Node start time", formatDateTime(m.getNodeStartTime))
                             t += ("Up time", X.timeSpan2HMSM(m.getUpTime))

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b81c6a25/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
index b9b5dbc..964de1d 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
@@ -17,36 +17,39 @@
 
 package org.apache.ignite.visor
 
-import java.io._
-import java.net._
-import java.text._
-import java.util.concurrent._
-import java.util.{HashSet => JHashSet, _}
-
 import org.apache.ignite.IgniteSystemProperties._
+import org.apache.ignite._
 import org.apache.ignite.cluster.{ClusterGroup, ClusterMetrics, ClusterNode}
 import org.apache.ignite.configuration.IgniteConfiguration
 import org.apache.ignite.events.EventType._
 import org.apache.ignite.events.{DiscoveryEvent, Event}
 import org.apache.ignite.internal.IgniteComponentType._
 import org.apache.ignite.internal.IgniteNodeAttributes._
+import org.apache.ignite.internal.IgniteVersionUtils._
 import org.apache.ignite.internal.cluster.ClusterGroupEmptyCheckedException
 import org.apache.ignite.internal.processors.spring.IgniteSpringProcessor
-import org.apache.ignite.internal.{IgniteVersionUtils, IgniteEx}
-import IgniteVersionUtils._
 import org.apache.ignite.internal.util.lang.{GridFunc => F}
 import org.apache.ignite.internal.util.typedef._
-import org.apache.ignite.internal.util.{IgniteUtils => U, GridConfigurationFinder}
+import org.apache.ignite.internal.util.{GridConfigurationFinder, IgniteUtils => U}
 import org.apache.ignite.internal.visor.VisorTaskArgument
 import org.apache.ignite.internal.visor.node.VisorNodeEventsCollectorTask
 import org.apache.ignite.internal.visor.node.VisorNodeEventsCollectorTask.VisorNodeEventsCollectorTaskArg
+import org.apache.ignite.internal.visor.util.VisorTaskUtils._
+import org.apache.ignite.internal.IgniteEx
 import org.apache.ignite.lang.{IgniteNotPeerDeployable, IgnitePredicate}
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi
 import org.apache.ignite.thread.IgniteThreadPoolExecutor
-import org.apache.ignite.visor.commands.{VisorConsoleCommand, VisorTextTable}
-import org.apache.ignite._
+
 import org.jetbrains.annotations.Nullable
 
+import java.io._
+import java.net._
+import java.text._
+import java.util.concurrent._
+import java.util.{HashSet => JHashSet, _}
+
+import org.apache.ignite.visor.commands.{VisorConsoleCommand, VisorTextTable}
+
 import scala.collection.JavaConversions._
 import scala.collection.immutable
 import scala.io.StdIn
@@ -1317,7 +1320,7 @@ object visor extends VisorTag {
             else {
                 val n = ignite.name
 
-                if (n == null) "<default>" else n
+                escapeName(n)
             }
         )
         t += ("Config path", safe(cfgPath))


[13/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237

Posted by ag...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237


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

Branch: refs/heads/ignite-45
Commit: 31180aa1760aa2500393e5fb284c28b4289330e4
Parents: eb14f3b 9e1377f
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Feb 26 18:42:30 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Feb 26 18:42:30 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/IgniteFs.java   |   2 +-
 .../ignite/events/DiscoveryCustomEvent.java     |  56 ----------
 .../org/apache/ignite/events/EventType.java     |  14 +--
 .../java/org/apache/ignite/igfs/package.html    |   2 +-
 .../internal/events/DiscoveryCustomEvent.java   |  68 ++++++++++++
 .../discovery/GridDiscoveryManager.java         |   7 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   9 +-
 .../internal/GridDiscoveryEventSelfTest.java    |   9 +-
 ...dStartupWithUndefinedIgniteHomeSelfTest.java | 103 +++++++++++++++++++
 .../testsuites/IgniteKernalSelfTestSuite.java   |   1 +
 .../java/org/apache/ignite/igfs/package.html    |   2 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   7 +-
 12 files changed, 199 insertions(+), 81 deletions(-)
----------------------------------------------------------------------



[34/50] incubator-ignite git commit: #ignite-237: remove test for consistent hash replicaCount.

Posted by ag...@apache.org.
#ignite-237: remove test for consistent hash replicaCount.


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

Branch: refs/heads/ignite-45
Commit: 5a76d70f99c276f95cfdb4ae97c09341fefc468a
Parents: 5f620d6
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Mar 2 12:49:56 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Mar 2 12:49:56 2015 +0300

----------------------------------------------------------------------
 .../impl/ClientPartitionAffinitySelfTest.java   | 323 -------------------
 .../client/suite/IgniteClientTestSuite.java     |   1 -
 2 files changed, 324 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5a76d70f/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
deleted file mode 100644
index 6af1c4c..0000000
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
+++ /dev/null
@@ -1,323 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.client.impl;
-
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.internal.client.*;
-import org.apache.ignite.testframework.*;
-import org.apache.ignite.testframework.junits.common.*;
-
-import java.util.*;
-
-/**
- * Client's partitioned affinity tests.
- */
-public class ClientPartitionAffinitySelfTest extends GridCommonAbstractTest {
-    /** Hash ID resolver. */
-    private static final GridClientPartitionAffinity.HashIdResolver HASH_ID_RSLVR =
-        new GridClientPartitionAffinity.HashIdResolver() {
-            @Override public Object getHashId(GridClientNode node) {
-                return node.nodeId();
-            }
-        };
-
-    /**
-     * Test predefined affinity - must be ported to other clients.
-     */
-    @SuppressWarnings("UnaryPlus")
-    public void testPredefined() throws Exception {
-        // Use Md5 hasher for this test.
-        GridClientPartitionAffinity aff = new GridClientPartitionAffinity();
-
-        getTestResources().inject(aff);
-
-        aff.setHashIdResolver(HASH_ID_RSLVR);
-
-        List<GridClientNode> nodes = new ArrayList<>();
-
-        nodes.add(createNode("000ea4cd-f449-4dcb-869a-5317c63bd619", 50));
-        nodes.add(createNode("010ea4cd-f449-4dcb-869a-5317c63bd62a", 60));
-        nodes.add(createNode("0209ec54-ff53-4fdb-8239-5a3ac1fb31bd", 70));
-        nodes.add(createNode("0309ec54-ff53-4fdb-8239-5a3ac1fb31ef", 80));
-        nodes.add(createNode("040c9b94-02ae-45a6-9d5c-a066dbdf2636", 90));
-        nodes.add(createNode("050c9b94-02ae-45a6-9d5c-a066dbdf2747", 100));
-        nodes.add(createNode("0601f916-4357-4cfe-a7df-49d4721690bf", 110));
-        nodes.add(createNode("0702f916-4357-4cfe-a7df-49d4721691c0", 120));
-
-        Map<Object, Integer> data = new LinkedHashMap<>();
-
-        data.put("", 4);
-        data.put("asdf", 4);
-        data.put("224ea4cd-f449-4dcb-869a-5317c63bd619", 5);
-        data.put("fdc9ec54-ff53-4fdb-8239-5a3ac1fb31bd", 2);
-        data.put("0f9c9b94-02ae-45a6-9d5c-a066dbdf2636", 2);
-        data.put("d8f1f916-4357-4cfe-a7df-49d4721690bf", 7);
-        data.put("c77ffeae-78a1-4ee6-a0fd-8d197a794412", 3);
-        data.put("35de9f21-3c9b-4f4a-a7d5-3e2c6cb01564", 1);
-        data.put("d67eb652-4e76-47fb-ad4e-cd902d9b868a", 7);
-
-        data.put(0, 4);
-        data.put(1, 7);
-        data.put(12, 5);
-        data.put(123, 6);
-        data.put(1234, 4);
-        data.put(12345, 6);
-        data.put(123456, 6);
-        data.put(1234567, 6);
-        data.put(12345678, 0);
-        data.put(123456789, 7);
-        data.put(1234567890, 7);
-        data.put(1234567890L, 7);
-        data.put(12345678901L, 2);
-        data.put(123456789012L, 1);
-        data.put(1234567890123L, 0);
-        data.put(12345678901234L, 1);
-        data.put(123456789012345L, 6);
-        data.put(1234567890123456L, 7);
-        data.put(-23456789012345L, 4);
-        data.put(-2345678901234L, 1);
-        data.put(-234567890123L, 5);
-        data.put(-23456789012L, 5);
-        data.put(-2345678901L, 7);
-        data.put(-234567890L, 4);
-        data.put(-234567890, 7);
-        data.put(-23456789, 7);
-        data.put(-2345678, 0);
-        data.put(-234567, 6);
-        data.put(-23456, 6);
-        data.put(-2345, 6);
-        data.put(-234, 7);
-        data.put(-23, 5);
-        data.put(-2, 4);
-
-        data.put(0x80000000, 4);
-        data.put(0x7fffffff, 7);
-        data.put(0x8000000000000000L, 4);
-        data.put(0x7fffffffffffffffL, 4);
-
-        data.put(+1.1, 3);
-        data.put(-10.01, 4);
-        data.put(+100.001, 4);
-        data.put(-1000.0001, 4);
-        data.put(+1.7976931348623157E+308, 6);
-        data.put(-1.7976931348623157E+308, 6);
-        data.put(+4.9E-324, 7);
-        data.put(-4.9E-324, 7);
-
-        boolean ok = true;
-
-        for (Map.Entry<Object, Integer> entry : data.entrySet()) {
-            UUID exp = nodes.get(entry.getValue()).nodeId();
-            UUID act = aff.node(entry.getKey(), nodes).nodeId();
-
-            if (exp.equals(act))
-                continue;
-
-            ok = false;
-
-            info("Failed to validate affinity for key '" + entry.getKey() + "' [expected=" + exp +
-                ", actual=" + act + ".");
-        }
-
-        if (ok)
-            return;
-
-        fail("Client partitioned affinity validation fails.");
-    }
-
-    /**
-     * Test predefined affinity - must be ported to other clients.
-     */
-    @SuppressWarnings("UnaryPlus")
-    public void testPredefinedHashIdResolver() throws Exception {
-        // Use Md5 hasher for this test.
-        GridClientPartitionAffinity aff = new GridClientPartitionAffinity();
-
-        getTestResources().inject(aff);
-
-        aff.setHashIdResolver(new GridClientPartitionAffinity.HashIdResolver() {
-            @Override public Object getHashId(GridClientNode node) {
-                return 1;
-            }
-        });
-
-        List<GridClientNode> nodes = new ArrayList<>();
-
-        nodes.add(createNode("000ea4cd-f449-4dcb-869a-5317c63bd619", 50));
-        nodes.add(createNode("010ea4cd-f449-4dcb-869a-5317c63bd62a", 60));
-        nodes.add(createNode("0209ec54-ff53-4fdb-8239-5a3ac1fb31bd", 70));
-        nodes.add(createNode("0309ec54-ff53-4fdb-8239-5a3ac1fb31ef", 80));
-        nodes.add(createNode("040c9b94-02ae-45a6-9d5c-a066dbdf2636", 90));
-        nodes.add(createNode("050c9b94-02ae-45a6-9d5c-a066dbdf2747", 100));
-        nodes.add(createNode("0601f916-4357-4cfe-a7df-49d4721690bf", 110));
-        nodes.add(createNode("0702f916-4357-4cfe-a7df-49d4721691c0", 120));
-
-        Map<Object, Integer> data = new LinkedHashMap<>();
-
-        data.put("", 4);
-        data.put("asdf", 3);
-        data.put("224ea4cd-f449-4dcb-869a-5317c63bd619", 5);
-        data.put("fdc9ec54-ff53-4fdb-8239-5a3ac1fb31bd", 2);
-        data.put("0f9c9b94-02ae-45a6-9d5c-a066dbdf2636", 2);
-        data.put("d8f1f916-4357-4cfe-a7df-49d4721690bf", 4);
-        data.put("c77ffeae-78a1-4ee6-a0fd-8d197a794412", 3);
-        data.put("35de9f21-3c9b-4f4a-a7d5-3e2c6cb01564", 4);
-        data.put("d67eb652-4e76-47fb-ad4e-cd902d9b868a", 2);
-
-        data.put(0, 4);
-        data.put(1, 1);
-        data.put(12, 7);
-        data.put(123, 1);
-        data.put(1234, 6);
-        data.put(12345, 2);
-        data.put(123456, 5);
-        data.put(1234567, 4);
-        data.put(12345678, 6);
-        data.put(123456789, 3);
-        data.put(1234567890, 3);
-        data.put(1234567890L, 3);
-        data.put(12345678901L, 0);
-        data.put(123456789012L, 1);
-        data.put(1234567890123L, 3);
-        data.put(12345678901234L, 5);
-        data.put(123456789012345L, 5);
-        data.put(1234567890123456L, 7);
-        data.put(-23456789012345L, 6);
-        data.put(-2345678901234L, 4);
-        data.put(-234567890123L, 3);
-        data.put(-23456789012L, 0);
-        data.put(-2345678901L, 4);
-        data.put(-234567890L, 5);
-        data.put(-234567890, 3);
-        data.put(-23456789, 3);
-        data.put(-2345678, 6);
-        data.put(-234567, 4);
-        data.put(-23456, 5);
-        data.put(-2345, 2);
-        data.put(-234, 7);
-        data.put(-23, 6);
-        data.put(-2, 6);
-
-        data.put(0x80000000, 7);
-        data.put(0x7fffffff, 1);
-        data.put(0x8000000000000000L, 7);
-        data.put(0x7fffffffffffffffL, 7);
-
-        data.put(+1.1, 2);
-        data.put(-10.01, 0);
-        data.put(+100.001, 2);
-        data.put(-1000.0001, 0);
-        data.put(+1.7976931348623157E+308, 6);
-        data.put(-1.7976931348623157E+308, 1);
-        data.put(+4.9E-324, 1);
-        data.put(-4.9E-324, 1);
-
-        boolean ok = true;
-
-        for (Map.Entry<Object, Integer> entry : data.entrySet()) {
-            UUID exp = nodes.get(entry.getValue()).nodeId();
-            UUID act = aff.node(entry.getKey(), nodes).nodeId();
-
-            if (exp.equals(act))
-                continue;
-
-            ok = false;
-
-            info("Failed to validate affinity for key '" + entry.getKey() + "' [expected=" + exp +
-                ", actual=" + act + ".");
-        }
-
-        if (ok)
-            return;
-
-        fail("Client partitioned affinity validation fails.");
-    }
-
-    /**
-     * Create node with specified node id and replica count.
-     *
-     * @param nodeId Node id.
-     * @param replicaCnt Node partitioned affinity replica count.
-     * @return New node with specified node id and replica count.
-     */
-    private GridClientNode createNode(String nodeId, int replicaCnt) {
-        return GridClientNodeImpl.builder()
-            .nodeId(UUID.fromString(nodeId))
-            .build();
-    }
-
-    /**
-     * Add {@code cnt} nodes into emulated topology.
-     *
-     * @param cnt Number of nodes to add into emulated topology.
-     * @param nodes Client topology.
-     * @param srvNodes Server topology.
-     */
-    private void addNodes(int cnt, Collection<GridClientNode> nodes, Collection<ClusterNode> srvNodes) {
-        while (cnt-- > 0) {
-            UUID nodeId = UUID.randomUUID();
-            int replicaCnt = (int)Math.round(Math.random() * 500) + 1;
-
-            nodes.add(GridClientNodeImpl.builder()
-                .nodeId(nodeId)
-                .build());
-
-            ClusterNode srvNode = new TestRichNode(nodeId);
-
-            srvNodes.add(srvNode);
-        }
-    }
-
-    /**
-     * Rich node stub to use in emulated server topology.
-     */
-    private static class TestRichNode extends GridTestNode {
-        /**
-         * Node id.
-         */
-        private final UUID nodeId;
-
-
-        /**
-         * Externalizable class requires public no-arg constructor.
-         */
-        @SuppressWarnings("UnusedDeclaration")
-        public TestRichNode() {
-            this(UUID.randomUUID());
-        }
-
-        /**
-         * Constructs rich node stub to use in emulated server topology.
-         *
-         * @param nodeId Node id.
-         */
-        private TestRichNode(UUID nodeId) {
-            this.nodeId = nodeId;
-        }
-
-        /** {@inheritDoc} */
-        @Override public UUID id() {
-            return nodeId;
-        }
-
-        /** {@inheritDoc} */
-        @Override public <T> T attribute(String name) {
-            return super.attribute(name);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5a76d70f/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java
index d89a960..4785248 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java
@@ -95,7 +95,6 @@ public class IgniteClientTestSuite extends TestSuite {
         suite.addTest(new TestSuite(ClientDefaultCacheSelfTest.class));
 
         suite.addTestSuite(ClientFutureAdapterSelfTest.class);
-        suite.addTestSuite(ClientPartitionAffinitySelfTest.class);
         suite.addTestSuite(ClientPropertiesConfigurationSelfTest.class);
         suite.addTestSuite(ClientConsistentHashSelfTest.class);
         suite.addTestSuite(ClientJavaHasherSelfTest.class);


[47/50] incubator-ignite git commit: IGNITE-45 - WIP

Posted by ag...@apache.org.
IGNITE-45 - WIP


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

Branch: refs/heads/ignite-45
Commit: c083c91deaa8b7061c1a0a43f700c349dce72023
Parents: 501bd5c
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Tue Mar 3 15:59:26 2015 -0800
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Tue Mar 3 15:59:26 2015 -0800

----------------------------------------------------------------------
 .../communication/GridIoMessageFactory.java     |  6 ++
 .../affinity/AffinityTopologyVersion.java       | 53 ++++++++++--
 .../GridCachePartitionExchangeManager.java      | 86 ++++++++++++--------
 .../processors/cache/GridCacheProcessor.java    |  7 +-
 4 files changed, 105 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c083c91d/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index 6109d74..a984142 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@ -22,6 +22,7 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.managers.checkpoint.*;
 import org.apache.ignite.internal.managers.deployment.*;
 import org.apache.ignite.internal.managers.eventstorage.*;
+import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
@@ -492,6 +493,11 @@ public class GridIoMessageFactory implements MessageFactory {
 
                 break;
 
+            case 89:
+                msg = new AffinityTopologyVersion();
+
+                break;
+
             default:
                 if (ext != null) {
                     for (MessageFactory factory : ext) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c083c91d/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
index be6fae5..12e3f8f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
@@ -17,23 +17,28 @@
 
 package org.apache.ignite.internal.processors.affinity;
 
+import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.plugin.extensions.communication.*;
 
 import java.io.*;
+import java.nio.*;
 
 /**
  *
  */
-public class AffinityTopologyVersion implements Comparable<AffinityTopologyVersion>, Externalizable {
+public class AffinityTopologyVersion implements Comparable<AffinityTopologyVersion>, Externalizable, Message {
     /** */
-    public static final AffinityTopologyVersion NONE = new AffinityTopologyVersion(-1);
+    public static final AffinityTopologyVersion NONE = new AffinityTopologyVersion(-1, 0);
 
     /** */
-    public static final AffinityTopologyVersion ZERO = new AffinityTopologyVersion(0);
+    public static final AffinityTopologyVersion ZERO = new AffinityTopologyVersion(0, 0);
 
     /** */
     private long topVer;
 
+    /** */
+    private int minorTopVer;
+
     /**
      * Empty constructor required by {@link Externalizable}.
      */
@@ -42,10 +47,14 @@ public class AffinityTopologyVersion implements Comparable<AffinityTopologyVersi
     }
 
     /**
-     * @param ver Version.
+     * @param topVer Version.
      */
-    public AffinityTopologyVersion(long ver) {
-        topVer = ver;
+    public AffinityTopologyVersion(
+        long topVer,
+        int minorTopVer
+    ) {
+        this.topVer = topVer;
+        this.minorTopVer = minorTopVer;
     }
 
     /**
@@ -66,7 +75,8 @@ public class AffinityTopologyVersion implements Comparable<AffinityTopologyVersi
      *
      */
     public AffinityTopologyVersion previous() {
-        return new AffinityTopologyVersion(topVer - 1);
+        // TODO IGNITE-45.
+        return new AffinityTopologyVersion(topVer - 1, 0);
     }
 
     /** {@inheritDoc} */
@@ -87,11 +97,36 @@ public class AffinityTopologyVersion implements Comparable<AffinityTopologyVersi
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         out.writeLong(topVer);
+        out.writeInt(minorTopVer);
     }
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         topVer = in.readLong();
+        minorTopVer = in.readInt();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        // TODO: implement.
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        // TODO: implement.
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return 89;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        // TODO: implement.
+        return 0;
     }
 
     /**
@@ -107,11 +142,11 @@ public class AffinityTopologyVersion implements Comparable<AffinityTopologyVersi
     public static AffinityTopologyVersion readFrom(MessageReader msgReader) {
         long topVer = msgReader.readLong("topVer.idx");
 
-        return new AffinityTopologyVersion(topVer);
+        return new AffinityTopologyVersion(topVer, 0);
     }
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return String.valueOf(topVer);
+        return S.toString(AffinityTopologyVersion.class, this);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c083c91d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 09edf52..76ecea4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -46,6 +46,7 @@ import java.util.concurrent.locks.*;
 import static java.util.concurrent.TimeUnit.*;
 import static org.apache.ignite.IgniteSystemProperties.*;
 import static org.apache.ignite.events.EventType.*;
+import static org.apache.ignite.internal.events.DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.*;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader.*;
 
@@ -82,6 +83,9 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     @GridToStringExclude
     private final ConcurrentMap<Integer, GridClientPartitionTopology<K, V>> clientTops = new ConcurrentHashMap8<>();
 
+    /** Minor topology version incremented each time a new dynamic cache is started. */
+    private volatile int minorTopVer;
+
     /** */
     private volatile GridDhtPartitionsExchangeFuture<K, V> lastInitializedFuture;
 
@@ -105,52 +109,61 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
             try {
                 ClusterNode loc = cctx.localNode();
 
-                assert e.type() == EVT_NODE_JOINED || e.type() == EVT_NODE_LEFT || e.type() == EVT_NODE_FAILED;
+                assert e.type() == EVT_NODE_JOINED || e.type() == EVT_NODE_LEFT || e.type() == EVT_NODE_FAILED ||
+                    e.type() == EVT_DISCOVERY_CUSTOM_EVT;
 
                 final ClusterNode n = e.eventNode();
 
-                assert !loc.id().equals(n.id());
+                if (e.type() != EVT_DISCOVERY_CUSTOM_EVT) {
+                    assert !loc.id().equals(n.id());
 
-                if (e.type() == EVT_NODE_LEFT || e.type() == EVT_NODE_FAILED) {
-                    assert cctx.discovery().node(n.id()) == null;
+                    if (e.type() == EVT_NODE_LEFT || e.type() == EVT_NODE_FAILED) {
+                        assert cctx.discovery().node(n.id()) == null;
 
-                    for (GridDhtPartitionsExchangeFuture<K, V> f : exchFuts.values())
-                        f.onNodeLeft(n.id());
-                }
+                        for (GridDhtPartitionsExchangeFuture<K, V> f : exchFuts.values())
+                            f.onNodeLeft(n.id());
+                    }
 
-                assert e.type() != EVT_NODE_JOINED || n.order() > loc.order() : "Node joined with smaller-than-local " +
-                    "order [newOrder=" + n.order() + ", locOrder=" + loc.order() + ']';
+                    assert
+                        e.type() != EVT_NODE_JOINED || n.order() > loc.order() :
+                        "Node joined with smaller-than-local " +
+                            "order [newOrder=" + n.order() + ", locOrder=" + loc.order() + ']';
 
-                GridDhtPartitionExchangeId exchId = exchangeId(n.id(), new AffinityTopologyVersion(e.topologyVersion()),
-                    e.type());
+                    GridDhtPartitionExchangeId exchId = exchangeId(n.id(),
+                        new AffinityTopologyVersion(e.topologyVersion(), minorTopVer = 0),
+                        e.type());
 
-                GridDhtPartitionsExchangeFuture<K, V> exchFut = exchangeFuture(exchId, e);
+                    GridDhtPartitionsExchangeFuture<K, V> exchFut = exchangeFuture(exchId, e);
 
-                // Start exchange process.
-                pendingExchangeFuts.add(exchFut);
+                    // Start exchange process.
+                    pendingExchangeFuts.add(exchFut);
 
-                // Event callback - without this callback future will never complete.
-                exchFut.onEvent(exchId, e);
+                    // Event callback - without this callback future will never complete.
+                    exchFut.onEvent(exchId, e);
 
-                if (log.isDebugEnabled())
-                    log.debug("Discovery event (will start exchange): " + exchId);
-
-                locExchFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
-                    @Override public void apply(IgniteInternalFuture<?> t) {
-                        if (!enterBusy())
-                            return;
-
-                        try {
-                            // Unwind in the order of discovery events.
-                            for (GridDhtPartitionsExchangeFuture<K, V> f = pendingExchangeFuts.poll(); f != null;
-                                f = pendingExchangeFuts.poll())
-                                addFuture(f);
-                        }
-                        finally {
-                            leaveBusy();
+                    if (log.isDebugEnabled())
+                        log.debug("Discovery event (will start exchange): " + exchId);
+
+                    locExchFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                        @Override public void apply(IgniteInternalFuture<?> t) {
+                            if (!enterBusy())
+                                return;
+
+                            try {
+                                // Unwind in the order of discovery events.
+                                for (GridDhtPartitionsExchangeFuture<K, V> f = pendingExchangeFuts.poll(); f != null;
+                                    f = pendingExchangeFuts.poll())
+                                    addFuture(f);
+                            }
+                            finally {
+                                leaveBusy();
+                            }
                         }
-                    }
-                });
+                    });
+                }
+                else {
+                    // TODO.
+                }
             }
             finally {
                 leaveBusy();
@@ -166,7 +179,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
         exchWorker = new ExchangeWorker();
 
-        cctx.gridEvents().addLocalEventListener(discoLsnr, EVT_NODE_JOINED, EVT_NODE_LEFT, EVT_NODE_FAILED);
+        cctx.gridEvents().addLocalEventListener(discoLsnr, EVT_NODE_JOINED, EVT_NODE_LEFT, EVT_NODE_FAILED,
+            EVT_DISCOVERY_CUSTOM_EVT);
 
         cctx.io().addHandler(0, GridDhtPartitionsSingleMessage.class,
             new MessageHandler<GridDhtPartitionsSingleMessage<K, V>>() {
@@ -200,7 +214,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
         assert startTime > 0;
 
-        final AffinityTopologyVersion startTopVer = new AffinityTopologyVersion(loc.order());
+        final AffinityTopologyVersion startTopVer = new AffinityTopologyVersion(loc.order(), minorTopVer);
 
         GridDhtPartitionExchangeId exchId = exchangeId(loc.id(), startTopVer, EVT_NODE_JOINED);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c083c91d/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 260cab0..069930e 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
@@ -25,7 +25,10 @@ import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.events.*;
+import org.apache.ignite.internal.managers.eventstorage.*;
 import org.apache.ignite.internal.processors.*;
 import org.apache.ignite.internal.processors.cache.datastructures.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
@@ -565,7 +568,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         ctx.discovery().setCustomEventListener(new GridPlainInClosure<Serializable>() {
             @Override public void apply(Serializable evt) {
                 if (evt instanceof DynamicCacheDescriptor)
-                    onCacheDeploymentRequested((DynamicCacheDescriptor)evt);
+                    onCacheStartRequested((DynamicCacheDescriptor)evt);
             }
         });
 
@@ -1290,7 +1293,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      *
      * @param startDesc Cache start descriptor.
      */
-    private void onCacheDeploymentRequested(DynamicCacheDescriptor startDesc) {
+    private void onCacheStartRequested(DynamicCacheDescriptor startDesc) {
         CacheConfiguration ccfg = startDesc.cacheConfiguration();
 
         // Check if cache with the same name was concurrently started form different node.


[48/50] incubator-ignite git commit: IGNITE-45 - WIP Rebuilt messages.

Posted by ag...@apache.org.
IGNITE-45 - WIP Rebuilt messages.


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

Branch: refs/heads/ignite-45
Commit: 55a9c501c4a643596a6ac0dada9dfbb7279bd8b7
Parents: c083c91
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Tue Mar 3 16:05:17 2015 -0800
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Tue Mar 3 16:05:17 2015 -0800

----------------------------------------------------------------------
 .../affinity/AffinityTopologyVersion.java       | 83 ++++++++++++++------
 .../cache/GridCacheEvictionRequest.java         |  4 +-
 .../distributed/GridCacheTtlUpdateRequest.java  |  4 +-
 .../dht/GridDhtAffinityAssignmentRequest.java   |  4 +-
 .../dht/GridDhtAffinityAssignmentResponse.java  |  4 +-
 .../distributed/dht/GridDhtLockRequest.java     |  4 +-
 .../distributed/dht/GridDhtTxFinishRequest.java |  4 +-
 .../dht/GridDhtTxPrepareRequest.java            |  4 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |  4 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |  4 +-
 .../dht/preloader/GridDhtForceKeysRequest.java  |  4 +-
 .../GridDhtPartitionDemandMessage.java          |  4 +-
 .../preloader/GridDhtPartitionExchangeId.java   |  4 +-
 .../preloader/GridDhtPartitionsFullMessage.java |  4 +-
 .../distributed/near/GridNearGetRequest.java    |  4 +-
 .../distributed/near/GridNearGetResponse.java   |  4 +-
 .../distributed/near/GridNearLockRequest.java   |  4 +-
 .../near/GridNearTxFinishRequest.java           |  4 +-
 .../near/GridNearTxPrepareRequest.java          |  4 +-
 19 files changed, 96 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55a9c501/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
index 12e3f8f..e276253 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
@@ -47,7 +47,15 @@ public class AffinityTopologyVersion implements Comparable<AffinityTopologyVersi
     }
 
     /**
-     * @param topVer Version.
+     * @param topVer Topology version.
+     */
+    public AffinityTopologyVersion(long topVer) {
+        this.topVer = topVer;
+    }
+
+    /**
+     * @param topVer Topology version.
+     * @param minorTopVer Minor topology version.
      */
     public AffinityTopologyVersion(
         long topVer,
@@ -108,14 +116,60 @@ public class AffinityTopologyVersion implements Comparable<AffinityTopologyVersi
 
     /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
-        // TODO: implement.
-        return false;
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeInt("minorTopVer", minorTopVer))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeLong("topVer", topVer))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
     }
 
     /** {@inheritDoc} */
     @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
-        // TODO: implement.
-        return false;
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                minorTopVer = reader.readInt("minorTopVer");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                topVer = reader.readLong("topVer");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return true;
     }
 
     /** {@inheritDoc} */
@@ -125,24 +179,7 @@ public class AffinityTopologyVersion implements Comparable<AffinityTopologyVersi
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        // TODO: implement.
-        return 0;
-    }
-
-    /**
-     * @param msgWriter Message writer.
-     */
-    public boolean writeTo(MessageWriter msgWriter) {
-        return msgWriter.writeLong("topVer.idx", topVer);
-    }
-
-    /**
-     * @param msgReader Message reader.
-     */
-    public static AffinityTopologyVersion readFrom(MessageReader msgReader) {
-        long topVer = msgReader.readLong("topVer.idx");
-
-        return new AffinityTopologyVersion(topVer, 0);
+        return 2;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55a9c501/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionRequest.java
index 8f32c33..1344495 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionRequest.java
@@ -169,7 +169,7 @@ public class GridCacheEvictionRequest<K, V> extends GridCacheMessage<K, V> imple
                 writer.incrementState();
 
             case 5:
-                if (!topVer.writeTo(writer))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -207,7 +207,7 @@ public class GridCacheEvictionRequest<K, V> extends GridCacheMessage<K, V> imple
                 reader.incrementState();
 
             case 5:
-                topVer = AffinityTopologyVersion.readFrom(reader);
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55a9c501/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTtlUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTtlUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTtlUpdateRequest.java
index 1e8ba38..89f46ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTtlUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTtlUpdateRequest.java
@@ -217,7 +217,7 @@ public class GridCacheTtlUpdateRequest<K, V> extends GridCacheMessage<K, V> {
                 writer.incrementState();
 
             case 6:
-                if (!topVer.writeTo(writer))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -275,7 +275,7 @@ public class GridCacheTtlUpdateRequest<K, V> extends GridCacheMessage<K, V> {
                 reader.incrementState();
 
             case 6:
-                topVer = AffinityTopologyVersion.readFrom(reader);
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55a9c501/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java
index 52fb062..4dc43e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java
@@ -89,7 +89,7 @@ public class GridDhtAffinityAssignmentRequest<K, V> extends GridCacheMessage<K,
 
         switch (writer.state()) {
             case 3:
-                if (!topVer.writeTo(writer))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -111,7 +111,7 @@ public class GridDhtAffinityAssignmentRequest<K, V> extends GridCacheMessage<K,
 
         switch (reader.state()) {
             case 3:
-                topVer = AffinityTopologyVersion.readFrom(reader);
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55a9c501/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java
index 2e32632..ff3eb8a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java
@@ -136,7 +136,7 @@ public class GridDhtAffinityAssignmentResponse<K, V> extends GridCacheMessage<K,
                 writer.incrementState();
 
             case 4:
-                if (!topVer.writeTo(writer))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -166,7 +166,7 @@ public class GridDhtAffinityAssignmentResponse<K, V> extends GridCacheMessage<K,
                 reader.incrementState();
 
             case 4:
-                topVer = AffinityTopologyVersion.readFrom(reader);
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55a9c501/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java
index c7f5d85..0e0cde7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java
@@ -401,7 +401,7 @@ public class GridDhtLockRequest<K, V> extends GridDistributedLockRequest<K, V> {
                 writer.incrementState();
 
             case 30:
-                if (!topVer.writeTo(writer))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -487,7 +487,7 @@ public class GridDhtLockRequest<K, V> extends GridDistributedLockRequest<K, V> {
                 reader.incrementState();
 
             case 30:
-                topVer = AffinityTopologyVersion.readFrom(reader);
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55a9c501/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java
index 3daace5..6d6446f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java
@@ -282,7 +282,7 @@ public class GridDhtTxFinishRequest<K, V> extends GridDistributedTxFinishRequest
                 writer.incrementState();
 
             case 26:
-                if (!topVer.writeTo(writer))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -370,7 +370,7 @@ public class GridDhtTxFinishRequest<K, V> extends GridDistributedTxFinishRequest
                 reader.incrementState();
 
             case 26:
-                topVer = AffinityTopologyVersion.readFrom(reader);
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55a9c501/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
index 8d894ec..c913137 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
@@ -396,7 +396,7 @@ public class GridDhtTxPrepareRequest<K, V> extends GridDistributedTxPrepareReque
                 writer.incrementState();
 
             case 34:
-                if (!topVer.writeTo(writer))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -506,7 +506,7 @@ public class GridDhtTxPrepareRequest<K, V> extends GridDistributedTxPrepareReque
                 reader.incrementState();
 
             case 34:
-                topVer = AffinityTopologyVersion.readFrom(reader);
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55a9c501/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
index 685c466..25c5862 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
@@ -779,7 +779,7 @@ public class GridDhtAtomicUpdateRequest<K, V> extends GridCacheMessage<K, V> imp
                 writer.incrementState();
 
             case 19:
-                if (!topVer.writeTo(writer))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -951,7 +951,7 @@ public class GridDhtAtomicUpdateRequest<K, V> extends GridCacheMessage<K, V> imp
                 reader.incrementState();
 
             case 19:
-                topVer = AffinityTopologyVersion.readFrom(reader);
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55a9c501/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java
index 7457b0b..f782402 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java
@@ -643,7 +643,7 @@ public class GridNearAtomicUpdateRequest<K, V> extends GridCacheMessage<K, V> im
                 writer.incrementState();
 
             case 19:
-                if (!topVer.writeTo(writer))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -813,7 +813,7 @@ public class GridNearAtomicUpdateRequest<K, V> extends GridCacheMessage<K, V> im
                 reader.incrementState();
 
             case 19:
-                topVer = AffinityTopologyVersion.readFrom(reader);
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55a9c501/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysRequest.java
index a17c5b74..fa6cd29 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysRequest.java
@@ -197,7 +197,7 @@ public class GridDhtForceKeysRequest<K, V> extends GridCacheMessage<K, V> implem
                 writer.incrementState();
 
             case 6:
-                if (!topVer.writeTo(writer))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -243,7 +243,7 @@ public class GridDhtForceKeysRequest<K, V> extends GridCacheMessage<K, V> implem
                 reader.incrementState();
 
             case 6:
-                topVer = AffinityTopologyVersion.readFrom(reader);
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55a9c501/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java
index 84b376a..b609197 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java
@@ -219,7 +219,7 @@ public class GridDhtPartitionDemandMessage<K, V> extends GridCacheMessage<K, V>
                 writer.incrementState();
 
             case 5:
-                if (!topVer.writeTo(writer))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -275,7 +275,7 @@ public class GridDhtPartitionDemandMessage<K, V> extends GridCacheMessage<K, V>
                 reader.incrementState();
 
             case 5:
-                topVer = AffinityTopologyVersion.readFrom(reader);
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55a9c501/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
index 71add90..d101efd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
@@ -173,7 +173,7 @@ public class GridDhtPartitionExchangeId implements Message, Comparable<GridDhtPa
                 writer.incrementState();
 
             case 2:
-                if (!topVer.writeTo(writer))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -208,7 +208,7 @@ public class GridDhtPartitionExchangeId implements Message, Comparable<GridDhtPa
                 reader.incrementState();
 
             case 2:
-                topVer = AffinityTopologyVersion.readFrom(reader);
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55a9c501/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
index b7cbf5e..fb30196 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
@@ -138,7 +138,7 @@ public class GridDhtPartitionsFullMessage<K, V> extends GridDhtPartitionsAbstrac
                 writer.incrementState();
 
             case 6:
-                if (!topVer.writeTo(writer))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -168,7 +168,7 @@ public class GridDhtPartitionsFullMessage<K, V> extends GridDhtPartitionsAbstrac
                 reader.incrementState();
 
             case 6:
-                topVer = AffinityTopologyVersion.readFrom(reader);
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55a9c501/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java
index 30126f8..bfe6768 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java
@@ -310,7 +310,7 @@ public class GridNearGetRequest<K, V> extends GridCacheMessage<K, V> implements
                 writer.incrementState();
 
             case 12:
-                if (!topVer.writeTo(writer))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -410,7 +410,7 @@ public class GridNearGetRequest<K, V> extends GridCacheMessage<K, V> implements
                 reader.incrementState();
 
             case 12:
-                topVer = AffinityTopologyVersion.readFrom(reader);
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55a9c501/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java
index 10b7291..f8013b6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java
@@ -247,7 +247,7 @@ public class GridNearGetResponse<K, V> extends GridCacheMessage<K, V> implements
                 writer.incrementState();
 
             case 8:
-                if (!topVer.writeTo(writer))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -315,7 +315,7 @@ public class GridNearGetResponse<K, V> extends GridCacheMessage<K, V> implements
                 reader.incrementState();
 
             case 8:
-                topVer = AffinityTopologyVersion.readFrom(reader);
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55a9c501/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java
index e8c071b..48608e6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java
@@ -404,7 +404,7 @@ public class GridNearLockRequest<K, V> extends GridDistributedLockRequest<K, V>
                 writer.incrementState();
 
             case 33:
-                if (!topVer.writeTo(writer))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -514,7 +514,7 @@ public class GridNearLockRequest<K, V> extends GridDistributedLockRequest<K, V>
                 reader.incrementState();
 
             case 33:
-                topVer = AffinityTopologyVersion.readFrom(reader);
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55a9c501/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java
index fc843df..eec5d58 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java
@@ -199,7 +199,7 @@ public class GridNearTxFinishRequest<K, V> extends GridDistributedTxFinishReques
                 writer.incrementState();
 
             case 24:
-                if (!topVer.writeTo(writer))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -261,7 +261,7 @@ public class GridNearTxFinishRequest<K, V> extends GridDistributedTxFinishReques
                 reader.incrementState();
 
             case 24:
-                topVer = AffinityTopologyVersion.readFrom(reader);
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55a9c501/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
index 4df0bb8..4cd82ef 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
@@ -313,7 +313,7 @@ public class GridNearTxPrepareRequest<K, V> extends GridDistributedTxPrepareRequ
                 writer.incrementState();
 
             case 32:
-                if (!topVer.writeTo(writer))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
@@ -407,7 +407,7 @@ public class GridNearTxPrepareRequest<K, V> extends GridDistributedTxPrepareRequ
                 reader.incrementState();
 
             case 32:
-                topVer = AffinityTopologyVersion.readFrom(reader);
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;


[23/50] incubator-ignite git commit: #ignite-237: remove old GridCacheAttributes.

Posted by ag...@apache.org.
#ignite-237: remove old GridCacheAttributes.


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

Branch: refs/heads/ignite-45
Commit: 936b8792b9bd6839c7620f8f1021dcd82952aef4
Parents: 0d70de8
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 18:31:48 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 18:31:48 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAttributes.java   | 24 --------------------
 .../processors/cache/GridCacheProcessor.java    |  8 -------
 2 files changed, 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/936b8792/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
index 8ecd84c..94ae1fa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
@@ -119,12 +119,6 @@ public class GridCacheAttributes implements Externalizable {
     private int affKeyBackups = -1;
 
     /** */
-    private int affReplicas = -1;
-
-    /** */
-    private String affReplicaCntAttrName;
-
-    /** */
     private String affHashIdRslvrClsName;
 
     /** */
@@ -294,13 +288,6 @@ public class GridCacheAttributes implements Externalizable {
     }
 
     /**
-     * @return Affinity replicas.
-     */
-    public int affinityReplicas() {
-        return affReplicas;
-    }
-
-    /**
      * @return Affinity partitions count.
      */
     public int affinityPartitionsCount() {
@@ -308,13 +295,6 @@ public class GridCacheAttributes implements Externalizable {
     }
 
     /**
-     * @return Aff replicas count attr name.
-     */
-    public String affinityReplicaCountAttrName() {
-        return affReplicaCntAttrName;
-    }
-
-    /**
      * @return Affinity hash ID resolver class name.
      */
     public String affinityHashIdResolverClassName() {
@@ -537,8 +517,6 @@ public class GridCacheAttributes implements Externalizable {
         out.writeBoolean(affInclNeighbors);
         out.writeInt(affKeyBackups);
         out.writeInt(affPartsCnt);
-        out.writeInt(affReplicas);
-        U.writeString(out, affReplicaCntAttrName);
         U.writeString(out, affHashIdRslvrClsName);
 
         U.writeString(out, evictFilterClsName);
@@ -581,8 +559,6 @@ public class GridCacheAttributes implements Externalizable {
         affInclNeighbors = in.readBoolean();
         affKeyBackups = in.readInt();
         affPartsCnt = in.readInt();
-        affReplicas = in.readInt();
-        affReplicaCntAttrName = U.readString(in);
         affHashIdRslvrClsName = U.readString(in);
 
         evictFilterClsName = U.readString(in);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/936b8792/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 f00a853..3e5e124 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
@@ -1186,14 +1186,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                                 "Affinity key backups", locAttr.affinityKeyBackups(),
                                 rmtAttr.affinityKeyBackups(), true);
 
-                            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "affinityReplicas",
-                                "Affinity replicas", locAttr.affinityReplicas(),
-                                rmtAttr.affinityReplicas(), true);
-
-                            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "affinityReplicaCountAttrName",
-                                "Affinity replica count attribute name", locAttr.affinityReplicaCountAttrName(),
-                                rmtAttr.affinityReplicaCountAttrName(), true);
-
                             CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "cacheAffinity.hashIdResolver",
                                 "Partitioned cache affinity hash ID resolver class",
                                 locAttr.affinityHashIdResolverClassName(), rmtAttr.affinityHashIdResolverClassName(),


[45/50] incubator-ignite git commit: Merge branch 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-45

Posted by ag...@apache.org.
Merge branch 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-45


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

Branch: refs/heads/ignite-45
Commit: fe2985b9ed475cc2bc51de81662fcb25f327eb1e
Parents: 38ebb3e 6909cc4
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Tue Mar 3 12:00:00 2015 -0800
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Tue Mar 3 12:00:00 2015 -0800

----------------------------------------------------------------------
 .../ClientAbstractMultiThreadedSelfTest.java    |  37 +-
 .../impl/ClientPartitionAffinitySelfTest.java   | 406 -----------
 .../ClientAbstractMultiNodeSelfTest.java        |  54 +-
 .../integration/ClientAbstractSelfTest.java     |   8 -
 .../ClientTcpUnreachableMultiNodeSelfTest.java  |   4 -
 .../client/suite/IgniteClientTestSuite.java     |   1 -
 .../src/test/resources/spring-server-node.xml   |   2 +-
 .../test/resources/spring-server-ssl-node.xml   |   2 +-
 .../java/org/apache/ignite/IgniteEvents.java    |   2 +-
 .../ignite/cache/CacheTypeFieldMetadata.java    |   2 +-
 .../apache/ignite/cache/CacheTypeMetadata.java  |  18 +-
 .../java/org/apache/ignite/cache/GridCache.java |   2 +-
 .../CacheConsistentHashAffinityFunction.java    | 702 -------------------
 .../cache/affinity/consistenthash/package.html  |  24 -
 .../CacheRendezvousAffinityFunction.java        |   2 +-
 .../configuration/CacheConfiguration.java       |   2 +-
 .../configuration/IgniteConfiguration.java      |   6 +-
 .../ignite/internal/client/GridClientNode.java  |   8 -
 .../client/GridClientPartitionAffinity.java     |   2 +-
 .../client/impl/GridClientNodeImpl.java         |  22 -
 .../connection/GridClientNioTcpConnection.java  |   3 +-
 .../processors/cache/GridCacheAttributes.java   |  32 +-
 .../processors/cache/GridCacheProcessor.java    |  29 +-
 .../rest/client/message/GridClientNodeBean.java |  23 -
 .../top/GridTopologyCommandHandler.java         |   8 -
 .../cache/VisorCacheAffinityConfiguration.java  |  19 +-
 .../config/load/dsi-49-server-production.xml    |   2 +-
 .../GridCacheAffinityBackupsSelfTest.java       |   9 -
 .../ignite/IgniteCacheAffinitySelfTest.java     |  30 +-
 ...AffinityProcessorConsistentHashSelfTest.java |  31 -
 ...idCacheConfigurationConsistencySelfTest.java |  46 +-
 ...ridCacheConfigurationValidationSelfTest.java |   8 +-
 .../cache/GridCacheOffHeapSelfTest.java         |  12 +-
 .../GridCachePartitionedAffinitySpreadTest.java |  16 +-
 .../IgniteClientAffinityAssignmentSelfTest.java |  16 +-
 .../GridCacheClientModesAbstractSelfTest.java   |   4 +-
 ...dCachePartitionedAffinityFilterSelfTest.java |   6 +-
 ...GridCachePreloadRestartAbstractSelfTest.java |   4 +-
 .../dht/GridCacheColocatedDebugTest.java        |   4 +-
 .../dht/GridCacheDhtEntrySelfTest.java          |   4 +-
 ...GridCacheDhtEvictionNearReadersSelfTest.java |  13 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |  13 +-
 .../dht/GridCacheDhtInternalEntrySelfTest.java  |   4 +-
 .../dht/GridCacheDhtPreloadBigDataSelfTest.java |   4 +-
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |   4 +-
 .../GridCacheDhtPreloadDisabledSelfTest.java    |   4 +-
 .../GridCacheDhtPreloadMessageCountTest.java    |   4 +-
 ...ridCacheDhtPreloadMultiThreadedSelfTest.java |   4 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |   4 +-
 .../GridCacheDhtPreloadStartStopSelfTest.java   |   4 +-
 .../dht/GridCacheDhtPreloadUnloadSelfTest.java  |   4 +-
 .../distributed/dht/GridCacheDhtTestUtils.java  |   6 +-
 ...dCachePartitionedTopologyChangeSelfTest.java |   7 +-
 ...ridCachePartitionedUnloadEventsSelfTest.java |   5 +-
 ...ffinityFunctionExcludeNeighborsSelfTest.java |  32 -
 ...AffinityExcludeNeighborsPerformanceTest.java |   4 +-
 ...rtitionedAffinityHashIdResolverSelfTest.java |   4 +-
 .../GridCachePartitionedAffinitySelfTest.java   | 226 ------
 .../GridCachePartitionedFullApiSelfTest.java    |  11 +-
 .../GridCachePartitionedNodeRestartTest.java    |   4 +-
 ...ePartitionedOptimisticTxNodeRestartTest.java |   4 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |   4 +-
 ...cheSynchronousEvictionsFailoverSelfTest.java |   4 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |   4 +-
 ...idOffHeapPartitionedMapAbstractSelfTest.java |   4 +-
 ...apPartitionedMapPerformanceAbstractTest.java |   4 +-
 ...GridUnsafePartitionedMapPerformanceTest.java |   4 +-
 .../capacity/spring-capacity-cache.xml          |   2 +-
 .../loadtests/colocation/spring-colocation.xml  |   2 +-
 .../GridCachePartitionedAtomicLongLoadTest.java |   4 +-
 ...apPartitionedMapPerformanceAbstractTest.java |   4 +-
 ...GridUnsafePartitionedMapPerformanceTest.java |   4 +-
 .../ignite/testframework/GridTestUtils.java     |  11 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 -
 .../ignite/testsuites/IgniteCacheTestSuite.java |   1 -
 .../hibernate/HibernateL2CacheSelfTest.java     |   8 +-
 .../processors/cache/GridCacheSwapSelfTest.java |  12 +-
 .../ignite/schema/generator/PojoGenerator.java  |   3 +-
 .../schema/generator/SnippetGenerator.java      | 138 ++++
 .../apache/ignite/schema/ui/SchemaLoadApp.java  |   2 +
 .../commands/cache/VisorCacheCommand.scala      |  18 +-
 .../commands/cache/VisorCacheScanCommand.scala  |   7 +-
 .../config/VisorConfigurationCommand.scala      |   5 +-
 .../visor/commands/node/VisorNodeCommand.scala  |   6 +-
 .../scala/org/apache/ignite/visor/visor.scala   |  27 +-
 85 files changed, 350 insertions(+), 1906 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fe2985b9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fe2985b9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java
index 82dc76a,e9b8a23..8be08de
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java
@@@ -20,9 -20,7 +20,8 @@@ package org.apache.ignite.internal.proc
  import org.apache.ignite.*;
  import org.apache.ignite.cache.*;
  import org.apache.ignite.cache.affinity.*;
- import org.apache.ignite.cache.affinity.consistenthash.*;
  import org.apache.ignite.cluster.*;
 +import org.apache.ignite.internal.processors.affinity.*;
  import org.apache.ignite.internal.processors.cache.*;
  import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.*;
  import org.apache.ignite.internal.util.typedef.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fe2985b9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
index 15b13c2,b8a3cc9..4b74816
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
@@@ -19,10 -19,9 +19,10 @@@ package org.apache.ignite.internal.proc
  
  import org.apache.ignite.*;
  import org.apache.ignite.cache.*;
- import org.apache.ignite.cache.affinity.consistenthash.*;
+ import org.apache.ignite.cache.affinity.rendezvous.*;
  import org.apache.ignite.configuration.*;
  import org.apache.ignite.internal.*;
 +import org.apache.ignite.internal.processors.affinity.*;
  import org.apache.ignite.internal.processors.cache.*;
  import org.apache.ignite.internal.processors.cache.transactions.*;
  import org.apache.ignite.internal.util.typedef.internal.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fe2985b9/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
index 3a245f7,0449e48..85a31ae
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
@@@ -901,8 -891,8 +892,8 @@@ public final class GridTestUtils 
              while (true) {
                  boolean wait = false;
  
-                 for (int p = 0; p < affinity(cache).partitions(); p++) {
+                 for (int p = 0; p < g.affinity(cacheName).partitions(); p++) {
 -                    Collection<ClusterNode> nodes = top.nodes(p, -1);
 +                    Collection<ClusterNode> nodes = top.nodes(p, AffinityTopologyVersion.NONE);
  
                      if (nodes.size() > backups + 1) {
                          LT.warn(log, null, "Partition map was not updated yet (will wait) [grid=" + g.name() +


[20/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237

Posted by ag...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237


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

Branch: refs/heads/ignite-45
Commit: bcf802d9a235b0a0e451bd2ac50ec0369e02f76b
Parents: 148ca09 dd66167
Author: sboikov <sb...@gridgain.com>
Authored: Fri Feb 27 16:56:52 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Feb 27 16:56:52 2015 +0300

----------------------------------------------------------------------
 config/hadoop/default-config.xml                |  12 +
 .../src/main/java/org/apache/ignite/Ignite.java |   4 +-
 .../configuration/QueryConfiguration.java       |  37 +-
 .../java/org/apache/ignite/igfs/IgfsMode.java   |   6 +-
 .../config/GridTestProperties.java              |  10 +-
 .../client/hadoop/GridHadoopClientProtocol.java |   6 +-
 .../hadoop/IgfsHadoopFileSystemWrapper.java     | 412 ++++++++++++++++++
 .../igfs/hadoop/v1/IgfsHadoopFileSystem.java    |   3 +-
 .../igfs/hadoop/v2/IgfsHadoopFileSystem.java    |   3 +-
 .../igfs/hadoop/IgfsHadoopFSProperties.java     |  10 +-
 .../hadoop/IgfsHadoopFileSystemWrapper.java     | 413 -------------------
 .../internal/igfs/hadoop/IgfsHadoopReader.java  |   2 +-
 .../internal/igfs/hadoop/IgfsHadoopUtils.java   |   4 +-
 .../hadoop/GridHadoopClassLoader.java           |  12 +-
 .../processors/hadoop/GridHadoopSetup.java      |   8 +-
 .../processors/hadoop/GridHadoopUtils.java      |   4 +-
 .../collections/GridHadoopHashMultimapBase.java |   2 +-
 .../GridHadoopExternalCommunication.java        |  14 +-
 .../hadoop/v1/GridHadoopV1MapTask.java          |   6 +-
 .../v2/GridHadoopV2JobResourceManager.java      |   2 +-
 .../GridHadoopClientProtocolSelfTest.java       |   6 +-
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |   2 +-
 .../IgfsHadoop20FileSystemAbstractSelfTest.java |   2 +-
 .../igfs/IgfsHadoopDualAbstractSelfTest.java    |   2 +-
 .../IgfsHadoopFileSystemAbstractSelfTest.java   |   1 +
 ...fsHadoopFileSystemSecondaryModeSelfTest.java |   2 +-
 .../hadoop/GridHadoopGroupingTest.java          |   4 +-
 .../igfs/IgfsPerformanceBenchmark.java          |   9 +-
 28 files changed, 530 insertions(+), 468 deletions(-)
----------------------------------------------------------------------



[33/50] incubator-ignite git commit: #ignite-239: remove replicaCount from GridClientNode and GridClientNodeBean.

Posted by ag...@apache.org.
#ignite-239: remove replicaCount from GridClientNode and GridClientNodeBean.


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

Branch: refs/heads/ignite-45
Commit: 5f620d62191188ff4dcbe55c5d0399603e494b45
Parents: 6c67bdf
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Mar 2 12:40:13 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Mar 2 12:40:13 2015 +0300

----------------------------------------------------------------------
 .../impl/ClientPartitionAffinitySelfTest.java   |  4 +---
 .../ClientTcpUnreachableMultiNodeSelfTest.java  |  4 ----
 .../ignite/internal/client/GridClientNode.java  |  8 -------
 .../client/GridClientPartitionAffinity.java     |  2 +-
 .../client/impl/GridClientNodeImpl.java         | 22 -------------------
 .../connection/GridClientNioTcpConnection.java  |  3 +--
 .../rest/client/message/GridClientNodeBean.java | 23 --------------------
 7 files changed, 3 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f620d62/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
index 7ece534..6af1c4c 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
@@ -152,7 +152,7 @@ public class ClientPartitionAffinitySelfTest extends GridCommonAbstractTest {
 
         aff.setHashIdResolver(new GridClientPartitionAffinity.HashIdResolver() {
             @Override public Object getHashId(GridClientNode node) {
-                return node.replicaCount();
+                return 1;
             }
         });
 
@@ -258,7 +258,6 @@ public class ClientPartitionAffinitySelfTest extends GridCommonAbstractTest {
     private GridClientNode createNode(String nodeId, int replicaCnt) {
         return GridClientNodeImpl.builder()
             .nodeId(UUID.fromString(nodeId))
-            .replicaCount(replicaCnt)
             .build();
     }
 
@@ -276,7 +275,6 @@ public class ClientPartitionAffinitySelfTest extends GridCommonAbstractTest {
 
             nodes.add(GridClientNodeImpl.builder()
                 .nodeId(nodeId)
-                .replicaCount(replicaCnt)
                 .build());
 
             ClusterNode srvNode = new TestRichNode(nodeId);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f620d62/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientTcpUnreachableMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientTcpUnreachableMultiNodeSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientTcpUnreachableMultiNodeSelfTest.java
index bb3710e..fda7e99 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientTcpUnreachableMultiNodeSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientTcpUnreachableMultiNodeSelfTest.java
@@ -85,10 +85,6 @@ public class ClientTcpUnreachableMultiNodeSelfTest extends ClientTcpMultiNodeSel
                     return node.caches();
                 }
 
-                @Override public int replicaCount() {
-                    return node.replicaCount();
-                }
-
                 @Override public List<String> tcpAddresses() {
                     return node.tcpAddresses();
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f620d62/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientNode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientNode.java b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientNode.java
index ac2fe86..74e612d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientNode.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientNode.java
@@ -99,14 +99,6 @@ public interface GridClientNode {
     public Map<String, GridClientCacheMode> caches();
 
     /**
-     * Gets node replica count for consistent hash ring (valid only for
-     * {@code PARTITIONED} caches).
-     *
-     * @return Node replica count for consistent hash ring.
-     */
-    public int replicaCount();
-
-    /**
      * Gets collection of addresses on which REST binary protocol is bound.
      *
      * @param proto Protocol for which addresses are obtained.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f620d62/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientPartitionAffinity.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientPartitionAffinity.java b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientPartitionAffinity.java
index a0fad61..b747bc4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientPartitionAffinity.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientPartitionAffinity.java
@@ -274,7 +274,7 @@ public class GridClientPartitionAffinity implements GridClientDataAffinity, Grid
         nodeInfo = new NodeInfo(nodeId, hashIdRslvr == null ? nodeId : hashIdRslvr.getHashId(n));
 
         addedNodes.put(nodeId, nodeInfo);
-        nodeHash.addNode(nodeInfo, n.replicaCount());
+        nodeHash.addNode(nodeInfo, 1);
 
         return nodeInfo;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f620d62/modules/core/src/main/java/org/apache/ignite/internal/client/impl/GridClientNodeImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/GridClientNodeImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/GridClientNodeImpl.java
index 6692550..9c59b84 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/GridClientNodeImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/GridClientNodeImpl.java
@@ -53,9 +53,6 @@ public class GridClientNodeImpl implements GridClientNode {
     /** Node caches. */
     private Map<String, GridClientCacheMode> caches = Collections.emptyMap();
 
-    /** Replica count for partitioned cache. */
-    private int replicaCnt;
-
     /** Connectable property. */
     private boolean connectable;
 
@@ -95,7 +92,6 @@ public class GridClientNodeImpl implements GridClientNode {
             .tcpAddresses(from.tcpAddresses())
             .tcpPort(from.tcpPort())
             .caches(from.caches())
-            .replicaCount(from.replicaCount())
             .connectable(from.connectable());
 
         if (!skipAttrs)
@@ -154,11 +150,6 @@ public class GridClientNodeImpl implements GridClientNode {
     }
 
     /** {@inheritDoc} */
-    @Override public int replicaCount() {
-        return replicaCnt;
-    }
-
-    /** {@inheritDoc} */
     @Override public Collection<InetSocketAddress> availableAddresses(GridClientProtocol proto,
         boolean filterResolved) {
         Collection<String> addrs;
@@ -368,19 +359,6 @@ public class GridClientNodeImpl implements GridClientNode {
             return this;
         }
 
-
-        /**
-         * Sets replica count for node on consistent hash ring.
-         *
-         * @param replicaCnt Replica count.
-         * @return This for chaining.
-         */
-        public Builder replicaCount(int replicaCnt) {
-            impl.replicaCnt = replicaCnt;
-
-            return this;
-        }
-
         /**
          * Sets connectable property.
          *

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f620d62/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
index d84bca5..44d10f0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
@@ -868,8 +868,7 @@ public class GridClientNioTcpConnection extends GridClientConnection {
             .nodeId(nodeBean.getNodeId())
             .consistentId(nodeBean.getConsistentId())
             .tcpAddresses(nodeBean.getTcpAddresses())
-            .tcpPort(nodeBean.getTcpPort())
-            .replicaCount(nodeBean.getReplicaCount());
+            .tcpPort(nodeBean.getTcpPort());
 
         Map<String, GridClientCacheMode> caches = new HashMap<>();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f620d62/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientNodeBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientNodeBean.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientNodeBean.java
index 6605a15..833abd9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientNodeBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientNodeBean.java
@@ -56,9 +56,6 @@ public class GridClientNodeBean implements Externalizable {
     /** Node caches. */
     private Map<String, String> caches;
 
-    /** Default replica count for partitioned cache. */
-    private int replicaCnt;
-
     /**
      * Gets node ID.
      *
@@ -209,24 +206,6 @@ public class GridClientNodeBean implements Externalizable {
     }
 
     /**
-     * Gets node replica count on consistent hash ring.
-     *
-     * @return Node replica count.
-     */
-    public int getReplicaCount() {
-        return replicaCnt;
-    }
-
-    /**
-     * Sets node replica count on consistent hash ring.
-     *
-     * @param replicaCnt Node replica count.
-     */
-    public void setReplicaCount(int replicaCnt) {
-        this.replicaCnt = replicaCnt;
-    }
-
-    /**
      * Sets REST binary protocol port.
      *
      * @param tcpPort Port on which REST binary protocol is bound.
@@ -257,7 +236,6 @@ public class GridClientNodeBean implements Externalizable {
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         out.writeInt(tcpPort);
         out.writeInt(0); // Jetty port.
-        out.writeInt(replicaCnt);
 
         U.writeString(out, dfltCacheMode);
 
@@ -279,7 +257,6 @@ public class GridClientNodeBean implements Externalizable {
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         tcpPort = in.readInt();
         in.readInt(); // Jetty port.
-        replicaCnt = in.readInt();
 
         dfltCacheMode = U.readString(in);
 


[15/50] incubator-ignite git commit: #ignite-237: remove test validating that client partitioned affinity and cache partitioned affinity produce the same result

Posted by ag...@apache.org.
#ignite-237: remove test validating that client partitioned affinity and cache partitioned affinity produce the same result


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

Branch: refs/heads/ignite-45
Commit: 4d54c502fcc48896fd2e4321f18ffa6ad1611656
Parents: 120711b
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 12:53:00 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 12:53:00 2015 +0300

----------------------------------------------------------------------
 .../impl/ClientPartitionAffinitySelfTest.java   | 66 --------------------
 1 file changed, 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d54c502/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
index 444a84c..dde345e 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
@@ -267,45 +267,6 @@ public class ClientPartitionAffinitySelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * Validate client partitioned affinity and cache partitioned affinity produce the same result.
-     *
-     * @throws Exception On any exception.
-     */
-    public void testReplicas() throws Exception {
-        // Emulate nodes in topology.
-        Collection<GridClientNode> nodes = new ArrayList<>();
-        Collection<ClusterNode> srvNodes = new ArrayList<>();
-
-        // Define affinities to test.
-        GridClientPartitionAffinity aff = new GridClientPartitionAffinity();
-
-        getTestResources().inject(aff);
-
-        aff.setHashIdResolver(HASH_ID_RSLVR);
-
-        CacheRendezvousAffinityFunction srvAff = new CacheRendezvousAffinityFunction();
-
-        getTestResources().inject(srvAff);
-
-        srvAff.setHashIdResolver(new CacheAffinityNodeIdHashResolver());
-
-        // Define keys to test affinity for.
-        Collection<String> keys = new ArrayList<>(
-            Arrays.asList("", "1", "12", "asdf", "Hadoop\u3092\u6bba\u3059"));
-
-        for (int i = 0; i < 10; i++)
-            keys.add(UUID.randomUUID().toString());
-
-        // Test affinity behaviour on different topologies.
-        for (int i = 0; i < 20; i++) {
-            addNodes(1 + (int)Math.round(Math.random() * 50), nodes, srvNodes);
-
-            for (String key : keys)
-                assertSameAffinity(key, aff, srvAff, nodes, srvNodes);
-        }
-    }
-
-    /**
      * Add {@code cnt} nodes into emulated topology.
      *
      * @param cnt Number of nodes to add into emulated topology.
@@ -329,33 +290,6 @@ public class ClientPartitionAffinitySelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * Compare server and client affinity for specified key in current topology.
-     *
-     * @param key Key to validate affinity for.
-     * @param aff Client affinity.
-     * @param srvAff Server affinity.
-     * @param nodes Client topology.
-     * @param srvNodes Server topology.
-     */
-    private void assertSameAffinity(Object key, GridClientDataAffinity aff, CacheAffinityFunction srvAff,
-        Collection<? extends GridClientNode> nodes, Collection<ClusterNode> srvNodes) {
-        GridClientNode node = aff.node(key, nodes);
-        int part = srvAff.partition(key);
-
-        CacheAffinityFunctionContext ctx = new GridCacheAffinityFunctionContextImpl(new ArrayList<>(srvNodes),
-            null, null, 1, 0);
-
-        ClusterNode srvNode = F.first(srvAff.assignPartitions(ctx).get(part));
-
-        if (node == null)
-            assertNull(srvNode);
-        else {
-            assertNotNull(srvNode);
-            assertEquals(node.nodeId(), srvNode.id());
-        }
-    }
-
-    /**
      * Rich node stub to use in emulated server topology.
      */
     private static class TestRichNode extends GridTestNode {


[11/50] incubator-ignite git commit: #ignite-237: remove unused imports.

Posted by ag...@apache.org.
#ignite-237: remove unused imports.


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

Branch: refs/heads/ignite-45
Commit: 75d960eb9b16d587193deb20d1347aefdb9cc4d9
Parents: d07931d
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Feb 26 18:30:35 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Feb 26 18:30:35 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheAttributes.java       | 1 -
 .../apache/ignite/internal/processors/cache/GridCacheProcessor.java | 1 -
 .../processors/rest/handlers/top/GridTopologyCommandHandler.java    | 1 -
 .../internal/visor/cache/VisorCacheAffinityConfiguration.java       | 1 -
 4 files changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/75d960eb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
index 1f8b0b4..571ae25 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.internal.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/75d960eb/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 66a751c..f00a853 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
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cache.affinity.fair.*;
 import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cache.store.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/75d960eb/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
index 4bcc566..7c2a15b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.rest.handlers.top;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/75d960eb/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
index 7aae6f3..c43dc1c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.visor.cache;
 
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;


[21/50] incubator-ignite git commit: # ignite-237

Posted by ag...@apache.org.
# ignite-237


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

Branch: refs/heads/ignite-45
Commit: e50d1eadc11651c9029b81ccd0636ea003f5b1a7
Parents: bcf802d
Author: sboikov <sb...@gridgain.com>
Authored: Fri Feb 27 17:12:25 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Feb 27 17:12:25 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/GridCacheAffinityBackupsSelfTest.java |  8 --------
 .../GridCacheContinuousQueryAbstractSelfTest.java       | 12 ++++++------
 2 files changed, 6 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e50d1ead/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java
index 88a3cc1..0758993 100644
--- a/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java
@@ -58,14 +58,6 @@ public class GridCacheAffinityBackupsSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void testConsistentHashBackups() throws Exception {
-        for (int i = 0; i < nodesCnt; i++)
-            checkBackups(i, new CacheRendezvousAffinityFunction());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     public void testRendezvousBackups() throws Exception {
         for (int i = 0; i < nodesCnt; i++)
             checkBackups(i, new CacheRendezvousAffinityFunction());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e50d1ead/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
index 0d9e189..d2b1786 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
@@ -165,12 +165,14 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
             Set<Cache.Entry<Object, Object>> entries = ((IgniteKernal) grid(i)).cache(null).entrySet();
 
             for (Cache.Entry entry : entries) {
-                info("Not removed entry " + grid(i).affinity(null).isPrimary(grid(i).localNode(), entry.getKey()));
-                info("Not removed entry " + grid(i).affinity(null).isBackup(grid(i).localNode(), entry.getKey()));
+                boolean primary = grid(i).affinity(null).isPrimary(grid(i).localNode(), entry.getKey());
+
+                boolean backup = grid(i).affinity(null).isBackup(grid(i).localNode(), entry.getKey());
+
+                info("Not removed entry [grid=" + i + ", primary=" + primary + ", backup=" + backup + ']');
+
                 allEmpty = false;
             }
-
-            info("Cache is not empty: " + ((IgniteKernal) grid(i)).cache(null).entrySet());
         }
 
         assertTrue(allEmpty);
@@ -844,8 +846,6 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
         finally {
             stopGrid("anotherGrid");
         }
-
-        awaitPartitionMapExchange();
     }
 
     /**


[06/50] incubator-ignite git commit: #ignite-237: change affinity function in tests.

Posted by ag...@apache.org.
#ignite-237: change affinity function in tests.


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

Branch: refs/heads/ignite-45
Commit: 262aba541504dde165aab6e0c7456879a42bcb22
Parents: 13d0627
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Feb 26 16:15:58 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Feb 26 16:15:58 2015 +0300

----------------------------------------------------------------------
 ...idCacheConfigurationConsistencySelfTest.java | 46 ++++++--------------
 ...ridCacheConfigurationValidationSelfTest.java |  8 ++--
 2 files changed, 17 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/262aba54/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
index 3bd42dc..311c36f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
@@ -20,8 +20,8 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cache.affinity.fair.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cache.eviction.*;
 import org.apache.ignite.cache.eviction.fifo.*;
 import org.apache.ignite.cache.eviction.lru.*;
@@ -308,14 +308,14 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
             new C1<CacheConfiguration, Void>() {
                 /** {@inheritDoc} */
                 @Override public Void apply(CacheConfiguration cfg) {
-                    cfg.setAffinity(new CacheConsistentHashAffinityFunction() {/*No-op.*/});
+                    cfg.setAffinity(new CacheRendezvousAffinityFunction() {/*No-op.*/});
                     return null;
                 }
             },
             new C1<CacheConfiguration, Void>() {
                 /** {@inheritDoc} */
                 @Override public Void apply(CacheConfiguration cfg) {
-                    cfg.setAffinity(new CacheConsistentHashAffinityFunction());
+                    cfg.setAffinity(new CacheRendezvousAffinityFunction());
                     return null;
                 }
             }
@@ -573,13 +573,13 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
 
         backups = 1;
 
-        aff = new CacheConsistentHashAffinityFunction(false, 100);
+        aff = new CacheRendezvousAffinityFunction(false, 100);
 
         startGrid(1);
 
         // 2nd grid with another affinity.
         // Check include neighbors.
-        aff = new CacheConsistentHashAffinityFunction(true, 100);
+        aff = new CacheRendezvousAffinityFunction(true, 100);
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
@@ -590,7 +590,7 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
         backups = 2;
 
         // Check backups.
-        aff = new CacheConsistentHashAffinityFunction(false, 100);
+        aff = new CacheRendezvousAffinityFunction(false, 100);
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
@@ -601,7 +601,7 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
         backups = 1;
 
         // Partitions count.
-        aff = new CacheConsistentHashAffinityFunction(false, 1000);
+        aff = new CacheRendezvousAffinityFunction(false, 1000);
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
@@ -609,28 +609,8 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
             }
         }, IgniteCheckedException.class, "Affinity partitions count mismatch");
 
-        // Replicas count.
-        aff = new CacheConsistentHashAffinityFunction(false, 100);
-        ((CacheConsistentHashAffinityFunction)aff).setDefaultReplicas(1024);
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                return startGrid(2);
-            }
-        }, IgniteCheckedException.class, "Affinity replicas mismatch");
-
-        // Replicas count attribute name.
-        aff = new CacheConsistentHashAffinityFunction(false, 100);
-        ((CacheConsistentHashAffinityFunction)aff).setReplicaCountAttributeName("attr_name");
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                return startGrid(2);
-            }
-        }, IgniteCheckedException.class, "Affinity replica count attribute name mismatch");
-
         // Different hash ID resolver.
-        CacheConsistentHashAffinityFunction aff0 = new CacheConsistentHashAffinityFunction(false, 100);
+        CacheRendezvousAffinityFunction aff0 = new CacheRendezvousAffinityFunction(false, 100);
 
         aff0.setHashIdResolver(new CacheAffinityNodeIdHashResolver());
 
@@ -725,7 +705,7 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
             /** {@inheritDoc} */
             @SuppressWarnings("unchecked")
             @Override public Void apply(CacheConfiguration cfg) {
-                cfg.setAffinity(new CacheConsistentHashAffinityFunction() {/*No-op.*/});
+                cfg.setAffinity(new CacheRendezvousAffinityFunction() {/*No-op.*/});
 
                 cfg.setEvictionPolicy(new CacheFifoEvictionPolicy());
 
@@ -743,7 +723,7 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
         initCache = new C1<CacheConfiguration, Void>() {
             /** {@inheritDoc} */
             @Override public Void apply(CacheConfiguration cfg) {
-                cfg.setAffinity(new CacheConsistentHashAffinityFunction());
+                cfg.setAffinity(new CacheRendezvousAffinityFunction());
 
                 cfg.setEvictionPolicy(new CacheLruEvictionPolicy());
 
@@ -948,7 +928,7 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
             }
         }, IgniteCheckedException.class, null);
 
-        aff = new CacheConsistentHashAffinityFunction(true); // Check cannot set 'excludeNeighbors' flag.
+        aff = new CacheRendezvousAffinityFunction(true); // Check cannot set 'excludeNeighbors' flag.
         backups = Integer.MAX_VALUE;
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
@@ -957,12 +937,12 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
             }
         }, IgniteCheckedException.class, null);
 
-        aff = new CacheConsistentHashAffinityFunction(false, 100);
+        aff = new CacheRendezvousAffinityFunction(false, 100);
 
         startGrid(1);
 
         // Try to start node with  different number of partitions.
-        aff = new CacheConsistentHashAffinityFunction(false, 200);
+        aff = new CacheRendezvousAffinityFunction(false, 200);
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/262aba54/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationValidationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationValidationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationValidationSelfTest.java
index 8e5474a..325d346 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationValidationSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationValidationSelfTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -83,7 +83,7 @@ public class GridCacheConfigurationValidationSelfTest extends GridCommonAbstract
         dfltCacheCfg.setCacheMode(PARTITIONED);
         dfltCacheCfg.setPreloadMode(ASYNC);
         dfltCacheCfg.setWriteSynchronizationMode(FULL_SYNC);
-        dfltCacheCfg.setAffinity(new CacheConsistentHashAffinityFunction());
+        dfltCacheCfg.setAffinity(new CacheRendezvousAffinityFunction());
 
         // Non-default cache configuration.
         CacheConfiguration namedCacheCfg = defaultCacheConfiguration();
@@ -92,7 +92,7 @@ public class GridCacheConfigurationValidationSelfTest extends GridCommonAbstract
         namedCacheCfg.setPreloadMode(ASYNC);
         namedCacheCfg.setWriteSynchronizationMode(FULL_SYNC);
         namedCacheCfg.setName(NON_DFLT_CACHE_NAME);
-        namedCacheCfg.setAffinity(new CacheConsistentHashAffinityFunction());
+        namedCacheCfg.setAffinity(new CacheRendezvousAffinityFunction());
 
         // Modify cache config according to test parameters.
         if (gridName.contains(WRONG_PRELOAD_MODE_GRID_NAME))
@@ -100,7 +100,7 @@ public class GridCacheConfigurationValidationSelfTest extends GridCommonAbstract
         else if (gridName.contains(WRONG_CACHE_MODE_GRID_NAME))
             dfltCacheCfg.setCacheMode(REPLICATED);
         else if (gridName.contains(WRONG_AFFINITY_GRID_NAME)) {
-            dfltCacheCfg.setAffinity(new CacheConsistentHashAffinityFunction() {
+            dfltCacheCfg.setAffinity(new CacheRendezvousAffinityFunction() {
                 // No-op. Just to have another class name.
             });
         }


[09/50] incubator-ignite git commit: #ignite-237: change javadoc.

Posted by ag...@apache.org.
#ignite-237: change javadoc.


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

Branch: refs/heads/ignite-45
Commit: 87586da9c755f343e5468806ac0302b108cc434d
Parents: 96ad6c1
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Feb 26 16:26:29 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Feb 26 16:26:29 2015 +0300

----------------------------------------------------------------------
 modules/core/src/main/java/org/apache/ignite/cache/GridCache.java  | 2 +-
 .../java/org/apache/ignite/configuration/CacheConfiguration.java   | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/87586da9/modules/core/src/main/java/org/apache/ignite/cache/GridCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/GridCache.java b/modules/core/src/main/java/org/apache/ignite/cache/GridCache.java
index 8bbea1f..1b4be2d 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/GridCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/GridCache.java
@@ -211,7 +211,7 @@ public interface GridCache<K, V> extends CacheProjection<K, V> {
      * the left nodes, and that nodes are restarted before
      * {@link CacheConfiguration#getPreloadPartitionedDelay() preloadDelay} expires. To place nodes
      * on the same place in consistent hash ring, use
-     * {@link org.apache.ignite.cache.affinity.consistenthash.CacheConsistentHashAffinityFunction#setHashIdResolver(org.apache.ignite.cache.affinity.CacheAffinityNodeHashResolver)} to make sure that
+     * {@link org.apache.ignite.cache.affinity.rendezvous.CacheRendezvousAffinityFunction#setHashIdResolver(CacheAffinityNodeHashResolver)} to make sure that
      * a node maps to the same hash ID if re-started.
      * <p>
      * See {@link org.apache.ignite.configuration.CacheConfiguration#getPreloadPartitionedDelay()} for more information on how to configure

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/87586da9/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 57583d4..224176f 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
@@ -1343,7 +1343,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * For better efficiency user should usually make sure that new nodes get placed on
      * the same place of consistent hash ring as the left nodes, and that nodes are
      * restarted before this delay expires. To place nodes on the same place in consistent hash ring,
-     * use {@link org.apache.ignite.cache.affinity.consistenthash.CacheConsistentHashAffinityFunction#setHashIdResolver(org.apache.ignite.cache.affinity.CacheAffinityNodeHashResolver)}
+     * use {@link org.apache.ignite.cache.affinity.rendezvous.CacheRendezvousAffinityFunction#setHashIdResolver(CacheAffinityNodeHashResolver)}
      * to make sure that a node maps to the same hash ID event if restarted. As an example,
      * node IP address and port combination may be used in this case.
      * <p>


[29/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237

Posted by ag...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237


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

Branch: refs/heads/ignite-45
Commit: b378307eaef5e41a5de0cff33c7265cec9729d61
Parents: f27e9d2 036bd71
Author: ivasilinets <va...@gmail.com>
Authored: Sun Mar 1 21:22:32 2015 +0300
Committer: ivasilinets <va...@gmail.com>
Committed: Sun Mar 1 21:22:32 2015 +0300

----------------------------------------------------------------------
 config/hadoop/default-config.xml                |   12 +
 config/ignite-log4j.xml                         |    2 +-
 .../ComputeFibonacciContinuationExample.java    |   15 +-
 .../datagrid/CacheContinuousQueryExample.java   |    2 +-
 .../store/CacheNodeWithStoreStartup.java        |    6 +-
 .../store/jdbc/CacheJdbcPersonStore.java        |  115 +-
 .../examples/ScalarContinuationExample.scala    |   15 +-
 ipc/shmem/Makefile.am                           |   15 +
 ipc/shmem/igniteshmem/Makefile.am               |   15 +
 ipc/shmem/include/Makefile.am                   |   15 +
 modules/clients/src/test/keystore/generate.sh   |   15 +-
 .../src/main/java/org/apache/ignite/Ignite.java |    4 +-
 .../main/java/org/apache/ignite/IgniteFs.java   |    2 +-
 .../ignite/cache/query/ContinuousQuery.java     |   18 +-
 .../apache/ignite/cache/store/CacheStore.java   |    4 +-
 .../ignite/cache/store/CacheStoreAdapter.java   |    2 +-
 .../ignite/cache/store/CacheStoreSession.java   |   17 +-
 .../apache/ignite/cluster/ClusterMetrics.java   |    2 +-
 .../configuration/QueryConfiguration.java       |   37 +-
 .../ignite/events/DiscoveryCustomEvent.java     |   56 -
 .../org/apache/ignite/events/EventType.java     |   14 +-
 .../java/org/apache/ignite/igfs/IgfsMode.java   |    6 +-
 .../java/org/apache/ignite/igfs/package.html    |    2 +-
 .../apache/ignite/internal/GridProperties.java  |   78 -
 .../ignite/internal/GridUpdateNotifier.java     |    2 +-
 .../apache/ignite/internal/IgniteKernal.java    |   18 +-
 .../ignite/internal/IgniteProperties.java       |   79 +
 .../ignite/internal/IgniteVersionUtils.java     |    8 +-
 .../internal/events/DiscoveryCustomEvent.java   |   71 +
 .../discovery/GridDiscoveryManager.java         |    7 +-
 .../processors/cache/GridCacheStoreManager.java |    6 +-
 .../processors/cache/IgniteCacheProxy.java      |    6 +-
 .../plugin/IgnitePluginProcessor.java           |   24 +
 .../cache/VisorCacheMetricsCollectorTask.java   |   10 +-
 .../visor/node/VisorBasicConfiguration.java     |   17 -
 .../node/VisorNodeEventsCollectorTask.java      |   10 +-
 .../internal/visor/node/VisorNodeGcTask.java    |   10 +-
 .../internal/visor/node/VisorNodePingTask.java  |   10 +-
 .../optimized-classnames.previous.properties    |   15 +
 .../optimized/optimized-classnames.properties   | 1565 +-----------------
 .../apache/ignite/plugin/PluginProvider.java    |    5 +
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |    9 +-
 .../TcpDiscoveryCustomEventMessage.java         |    3 +
 .../internal/GridDiscoveryEventSelfTest.java    |    9 +-
 .../internal/GridUpdateNotifierSelfTest.java    |    2 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |    8 +-
 ...dStartupWithUndefinedIgniteHomeSelfTest.java |  103 ++
 .../config/GridTestProperties.java              |   10 +-
 .../junits/cache/TestCacheSession.java          |    5 +
 .../cache/TestThreadLocalCacheSession.java      |    5 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    1 +
 modules/extdata/p2p/pom.xml                     |    6 -
 modules/hadoop/pom.xml                          |   10 -
 .../client/hadoop/GridHadoopClientProtocol.java |    6 +-
 .../hadoop/IgfsHadoopFileSystemWrapper.java     |  412 +++++
 .../igfs/hadoop/v1/IgfsHadoopFileSystem.java    |    3 +-
 .../igfs/hadoop/v2/IgfsHadoopFileSystem.java    |    3 +-
 .../java/org/apache/ignite/igfs/package.html    |    2 +-
 .../igfs/hadoop/IgfsHadoopFSProperties.java     |   10 +-
 .../hadoop/IgfsHadoopFileSystemWrapper.java     |  413 -----
 .../internal/igfs/hadoop/IgfsHadoopReader.java  |    2 +-
 .../internal/igfs/hadoop/IgfsHadoopUtils.java   |    4 +-
 .../hadoop/GridHadoopClassLoader.java           |   12 +-
 .../processors/hadoop/GridHadoopSetup.java      |    8 +-
 .../processors/hadoop/GridHadoopUtils.java      |    4 +-
 .../collections/GridHadoopHashMultimapBase.java |    2 +-
 .../GridHadoopExternalCommunication.java        |   14 +-
 .../hadoop/v1/GridHadoopV1MapTask.java          |    6 +-
 .../v2/GridHadoopV2JobResourceManager.java      |    2 +-
 .../GridHadoopClientProtocolSelfTest.java       |    6 +-
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |    2 +-
 .../IgfsHadoop20FileSystemAbstractSelfTest.java |    2 +-
 .../igfs/IgfsHadoopDualAbstractSelfTest.java    |    2 +-
 .../IgfsHadoopFileSystemAbstractSelfTest.java   |    1 +
 ...fsHadoopFileSystemSecondaryModeSelfTest.java |    2 +-
 .../hadoop/GridHadoopGroupingTest.java          |    4 +-
 .../igfs/IgfsPerformanceBenchmark.java          |    9 +-
 .../testsuites/IgniteHadoopTestSuite.java       |    7 +-
 modules/hibernate/pom.xml                       |    6 -
 .../HibernateReadWriteAccessStrategy.java       |   81 +-
 modules/indexing/pom.xml                        |    6 -
 modules/jta/pom.xml                             |    6 -
 modules/scalar/pom.xml                          |    6 -
 modules/spring/pom.xml                          |    6 -
 modules/visor-console/pom.xml                   |    7 -
 .../commands/alert/VisorAlertCommand.scala      |    8 +-
 .../commands/cache/VisorCacheCommand.scala      |   82 +-
 .../config/VisorConfigurationCommand.scala      |  140 +-
 .../commands/disco/VisorDiscoveryCommand.scala  |    2 +-
 .../scala/org/apache/ignite/visor/visor.scala   |   64 +-
 .../commands/tasks/VisorTasksCommandSpec.scala  |    2 +-
 modules/web/pom.xml                             |    6 -
 modules/winservice/IgniteService.sln            |    2 +-
 .../IgniteService/IgniteService.csproj          |    2 +-
 .../config/benchmark-atomic-win.properties      |   15 +
 .../config/benchmark-atomic.properties          |   15 +
 .../config/benchmark-compute-win.properties     |   15 +
 .../config/benchmark-compute.properties         |   15 +
 .../config/benchmark-multicast.properties       |   15 +
 .../config/benchmark-query-win.properties       |   15 +
 .../yardstick/config/benchmark-query.properties |   15 +
 .../config/benchmark-tx-win.properties          |   15 +
 .../yardstick/config/benchmark-tx.properties    |   15 +
 .../yardstick/config/benchmark-win.properties   |   15 +
 modules/yardstick/config/benchmark.properties   |   15 +
 pom.xml                                         |  150 +-
 106 files changed, 1568 insertions(+), 2614 deletions(-)
----------------------------------------------------------------------



[25/50] incubator-ignite git commit: #ignite-237: move CacheConsistentHashAffinityFunction.

Posted by ag...@apache.org.
#ignite-237: move CacheConsistentHashAffinityFunction.


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

Branch: refs/heads/ignite-45
Commit: e3782644789c565162870fb2a41bd753e7631922
Parents: 414410b
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 19:36:34 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 19:36:34 2015 +0300

----------------------------------------------------------------------
 .../CacheConsistentHashAffinityFunction.java    | 703 -------------------
 1 file changed, 703 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e3782644/modules/core/src/main/java/org/gridgain/benchmarks/risk/affinity/CacheConsistentHashAffinityFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/benchmarks/risk/affinity/CacheConsistentHashAffinityFunction.java b/modules/core/src/main/java/org/gridgain/benchmarks/risk/affinity/CacheConsistentHashAffinityFunction.java
deleted file mode 100644
index 35be9b8..0000000
--- a/modules/core/src/main/java/org/gridgain/benchmarks/risk/affinity/CacheConsistentHashAffinityFunction.java
+++ /dev/null
@@ -1,703 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.gridgain.benchmarks.risk.affinity;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.tostring.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.resources.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-/**
- * Affinity function for partitioned cache. This function supports the following
- * configuration:
- * <ul>
- * <li>
- *      {@code backups} - Use this flag to control how many back up nodes will be
- *      assigned to every key. The default value is {@code 0}.
- * </li>
- * <li>
- *      {@code replicas} - Generally the more replicas a node gets, the more key assignments
- *      it will receive. You can configure different number of replicas for a node by
- *      setting user attribute with name {@link #getReplicaCountAttributeName()} to some
- *      number. Default value is {@code 512} defined by {@link #DFLT_REPLICA_COUNT} constant.
- * </li>
- * <li>
- *      {@code backupFilter} - Optional filter for back up nodes. If provided, then only
- *      nodes that pass this filter will be selected as backup nodes. If not provided, then
- *      primary and backup nodes will be selected out of all nodes available for this cache.
- * </li>
- * </ul>
- * <p>
- * Cache affinity can be configured for individual caches via {@link org.apache.ignite.configuration.CacheConfiguration#getAffinity()} method.
- */
-@Deprecated
-public class CacheConsistentHashAffinityFunction implements CacheAffinityFunction {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Flag to enable/disable consistency check (for internal use only). */
-    private static final boolean AFFINITY_CONSISTENCY_CHECK = Boolean.getBoolean("IGNITE_AFFINITY_CONSISTENCY_CHECK");
-
-    /** Default number of partitions. */
-    public static final int DFLT_PARTITION_COUNT = 10000;
-
-    /** Default replica count for partitioned caches. */
-    public static final int DFLT_REPLICA_COUNT = 128;
-
-    /**
-     * Name of node attribute to specify number of replicas for a node.
-     * Default value is {@code gg:affinity:node:replicas}.
-     */
-    public static final String DFLT_REPLICA_COUNT_ATTR_NAME = "gg:affinity:node:replicas";
-
-    /** Node hash. */
-    private transient GridConsistentHash<NodeInfo> nodeHash;
-
-    /** Total number of partitions. */
-    private int parts = DFLT_PARTITION_COUNT;
-
-    /** */
-    private int replicas = DFLT_REPLICA_COUNT;
-
-    /** */
-    private String attrName = DFLT_REPLICA_COUNT_ATTR_NAME;
-
-    /** */
-    private boolean exclNeighbors;
-
-    /**
-     * Optional backup filter. First node passed to this filter is primary node,
-     * and second node is a node being tested.
-     */
-    private IgniteBiPredicate<ClusterNode, ClusterNode> backupFilter;
-
-    /** */
-    private CacheAffinityNodeHashResolver hashIdRslvr = new CacheAffinityNodeAddressHashResolver();
-
-    /** Injected grid. */
-    @IgniteInstanceResource
-    private Ignite ignite;
-
-    /** Injected cache name. */
-    @CacheNameResource
-    private String cacheName;
-
-    /** Injected logger. */
-    @LoggerResource
-    private IgniteLogger log;
-
-    /** Initialization flag. */
-    @SuppressWarnings("TransientFieldNotInitialized")
-    private transient AtomicBoolean init = new AtomicBoolean();
-
-    /** Latch for initializing. */
-    @SuppressWarnings({"TransientFieldNotInitialized"})
-    private transient CountDownLatch initLatch = new CountDownLatch(1);
-
-    /** Nodes IDs. */
-    @GridToStringInclude
-    @SuppressWarnings({"TransientFieldNotInitialized"})
-    private transient ConcurrentMap<UUID, NodeInfo> addedNodes = new ConcurrentHashMap<>();
-
-    /** Optional backup filter. */
-    @GridToStringExclude
-    private final IgniteBiPredicate<NodeInfo, NodeInfo> backupIdFilter = new IgniteBiPredicate<NodeInfo, NodeInfo>() {
-        @Override public boolean apply(NodeInfo primaryNodeInfo, NodeInfo nodeInfo) {
-            return backupFilter == null || backupFilter.apply(primaryNodeInfo.node(), nodeInfo.node());
-        }
-    };
-
-    /** Map of neighbors. */
-    @SuppressWarnings("TransientFieldNotInitialized")
-    private transient ConcurrentMap<UUID, Collection<UUID>> neighbors =
-        new ConcurrentHashMap8<>();
-
-    /**
-     * Empty constructor with all defaults.
-     */
-    public CacheConsistentHashAffinityFunction() {
-        // No-op.
-    }
-
-    /**
-     * Initializes affinity with flag to exclude same-host-neighbors from being backups of each other
-     * and specified number of backups.
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code #getBackupFilter()} is set.
-     *
-     * @param exclNeighbors {@code True} if nodes residing on the same host may not act as backups
-     *      of each other.
-     */
-    public CacheConsistentHashAffinityFunction(boolean exclNeighbors) {
-        this.exclNeighbors = exclNeighbors;
-    }
-
-    /**
-     * Initializes affinity with flag to exclude same-host-neighbors from being backups of each other,
-     * and specified number of backups and partitions.
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code #getBackupFilter()} is set.
-     *
-     * @param exclNeighbors {@code True} if nodes residing on the same host may not act as backups
-     *      of each other.
-     * @param parts Total number of partitions.
-     */
-    public CacheConsistentHashAffinityFunction(boolean exclNeighbors, int parts) {
-        A.ensure(parts != 0, "parts != 0");
-
-        this.exclNeighbors = exclNeighbors;
-        this.parts = parts;
-    }
-
-    /**
-     * Initializes optional counts for replicas and backups.
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code backupFilter} is set.
-     *
-     * @param parts Total number of partitions.
-     * @param backupFilter Optional back up filter for nodes. If provided, backups will be selected
-     *      from all nodes that pass this filter. First argument for this filter is primary node, and second
-     *      argument is node being tested.
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code backupFilter} is set.
-     */
-    public CacheConsistentHashAffinityFunction(int parts,
-                                               @Nullable IgniteBiPredicate<ClusterNode, ClusterNode> backupFilter) {
-        A.ensure(parts != 0, "parts != 0");
-
-        this.parts = parts;
-        this.backupFilter = backupFilter;
-    }
-
-    /**
-     * Gets default count of virtual replicas in consistent hash ring.
-     * <p>
-     * To determine node replicas, node attribute with {@link #getReplicaCountAttributeName()}
-     * name will be checked first. If it is absent, then this value will be used.
-     *
-     * @return Count of virtual replicas in consistent hash ring.
-     */
-    public int getDefaultReplicas() {
-        return replicas;
-    }
-
-    /**
-     * Sets default count of virtual replicas in consistent hash ring.
-     * <p>
-     * To determine node replicas, node attribute with {@link #getReplicaCountAttributeName} name
-     * will be checked first. If it is absent, then this value will be used.
-     *
-     * @param replicas Count of virtual replicas in consistent hash ring.s
-     */
-    public void setDefaultReplicas(int replicas) {
-        this.replicas = replicas;
-    }
-
-    /**
-     * Gets total number of key partitions. To ensure that all partitions are
-     * equally distributed across all nodes, please make sure that this
-     * number is significantly larger than a number of nodes. Also, partition
-     * size should be relatively small. Try to avoid having partitions with more
-     * than quarter million keys.
-     * <p>
-     * Note that for fully replicated caches this method should always
-     * return {@code 1}.
-     *
-     * @return Total partition count.
-     */
-    public int getPartitions() {
-        return parts;
-    }
-
-    /**
-     * Sets total number of partitions.
-     *
-     * @param parts Total number of partitions.
-     */
-    public void setPartitions(int parts) {
-        this.parts = parts;
-    }
-
-    /**
-     * Gets hash ID resolver for nodes. This resolver is used to provide
-     * alternate hash ID, other than node ID.
-     * <p>
-     * Node IDs constantly change when nodes get restarted, which causes them to
-     * be placed on different locations in the hash ring, and hence causing
-     * repartitioning. Providing an alternate hash ID, which survives node restarts,
-     * puts node on the same location on the hash ring, hence minimizing required
-     * repartitioning.
-     *
-     * @return Hash ID resolver.
-     */
-    public CacheAffinityNodeHashResolver getHashIdResolver() {
-        return hashIdRslvr;
-    }
-
-    /**
-     * Sets hash ID resolver for nodes. This resolver is used to provide
-     * alternate hash ID, other than node ID.
-     * <p>
-     * Node IDs constantly change when nodes get restarted, which causes them to
-     * be placed on different locations in the hash ring, and hence causing
-     * repartitioning. Providing an alternate hash ID, which survives node restarts,
-     * puts node on the same location on the hash ring, hence minimizing required
-     * repartitioning.
-     *
-     * @param hashIdRslvr Hash ID resolver.
-     */
-    public void setHashIdResolver(CacheAffinityNodeHashResolver hashIdRslvr) {
-        this.hashIdRslvr = hashIdRslvr;
-    }
-
-    /**
-     * Gets optional backup filter. If not {@code null}, backups will be selected
-     * from all nodes that pass this filter. First node passed to this filter is primary node,
-     * and second node is a node being tested.
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code backupFilter} is set.
-     *
-     * @return Optional backup filter.
-     */
-    @Nullable public IgniteBiPredicate<ClusterNode, ClusterNode> getBackupFilter() {
-        return backupFilter;
-    }
-
-    /**
-     * Sets optional backup filter. If provided, then backups will be selected from all
-     * nodes that pass this filter. First node being passed to this filter is primary node,
-     * and second node is a node being tested.
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code backupFilter} is set.
-     *
-     * @param backupFilter Optional backup filter.
-     */
-    public void setBackupFilter(@Nullable IgniteBiPredicate<ClusterNode, ClusterNode> backupFilter) {
-        this.backupFilter = backupFilter;
-    }
-
-    /**
-     * Gets optional attribute name for replica count. If not provided, the
-     * default is {@link #DFLT_REPLICA_COUNT_ATTR_NAME}.
-     *
-     * @return User attribute name for replica count for a node.
-     */
-    public String getReplicaCountAttributeName() {
-        return attrName;
-    }
-
-    /**
-     * Sets optional attribute name for replica count. If not provided, the
-     * default is {@link #DFLT_REPLICA_COUNT_ATTR_NAME}.
-     *
-     * @param attrName User attribute name for replica count for a node.
-     */
-    public void setReplicaCountAttributeName(String attrName) {
-        this.attrName = attrName;
-    }
-
-    /**
-     * Checks flag to exclude same-host-neighbors from being backups of each other (default is {@code false}).
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code #getBackupFilter()} is set.
-     *
-     * @return {@code True} if nodes residing on the same host may not act as backups of each other.
-     */
-    public boolean isExcludeNeighbors() {
-        return exclNeighbors;
-    }
-
-    /**
-     * Sets flag to exclude same-host-neighbors from being backups of each other (default is {@code false}).
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code #getBackupFilter()} is set.
-     *
-     * @param exclNeighbors {@code True} if nodes residing on the same host may not act as backups of each other.
-     */
-    public void setExcludeNeighbors(boolean exclNeighbors) {
-        this.exclNeighbors = exclNeighbors;
-    }
-
-    /**
-     * Gets neighbors for a node.
-     *
-     * @param node Node.
-     * @return Neighbors.
-     */
-    private Collection<UUID> neighbors(final ClusterNode node) {
-        Collection<UUID> ns = neighbors.get(node.id());
-
-        if (ns == null) {
-            Collection<ClusterNode> nodes = ignite.cluster().forHost(node).nodes();
-
-            ns = F.addIfAbsent(neighbors, node.id(), new ArrayList<>(F.nodeIds(nodes)));
-        }
-
-        return ns;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public List<List<ClusterNode>> assignPartitions(CacheAffinityFunctionContext ctx) {
-        List<List<ClusterNode>> res = new ArrayList<>(parts);
-
-        Collection<ClusterNode> topSnapshot = ctx.currentTopologySnapshot();
-
-        for (int part = 0; part < parts; part++) {
-            res.add(F.isEmpty(topSnapshot) ?
-                Collections.<ClusterNode>emptyList() :
-                // Wrap affinity nodes with unmodifiable list since unmodifiable generic collection
-                // doesn't provide equals and hashCode implementations.
-                U.sealList(nodes(part, topSnapshot, ctx.backups())));
-        }
-
-        return res;
-    }
-
-    /**
-     * Assigns nodes to one partition.
-     *
-     * @param part Partition to assign nodes for.
-     * @param nodes Cache topology nodes.
-     * @return Assigned nodes, first node is primary, others are backups.
-     */
-    public Collection<ClusterNode> nodes(int part, Collection<ClusterNode> nodes, int backups) {
-        if (nodes == null)
-            return Collections.emptyList();
-
-        int nodesSize = nodes.size();
-
-        if (nodesSize == 0)
-            return Collections.emptyList();
-
-        if (nodesSize == 1) // Minor optimization.
-            return nodes;
-
-        initialize();
-
-        final Map<NodeInfo, ClusterNode> lookup = new GridLeanMap<>(nodesSize);
-
-        // Store nodes in map for fast lookup.
-        for (ClusterNode n : nodes)
-            // Add nodes into hash circle, if absent.
-            lookup.put(resolveNodeInfo(n), n);
-
-        Collection<NodeInfo> selected;
-
-        if (backupFilter != null) {
-            final IgnitePredicate<NodeInfo> p = new P1<NodeInfo>() {
-                @Override public boolean apply(NodeInfo id) {
-                    return lookup.containsKey(id);
-                }
-            };
-
-            final NodeInfo primaryId = nodeHash.node(part, p);
-
-            IgnitePredicate<NodeInfo> backupPrimaryIdFilter = new IgnitePredicate<NodeInfo>() {
-                @Override public boolean apply(NodeInfo node) {
-                    return backupIdFilter.apply(primaryId, node);
-                }
-            };
-
-            Collection<NodeInfo> backupIds = nodeHash.nodes(part, backups, p, backupPrimaryIdFilter);
-
-            if (F.isEmpty(backupIds) && primaryId != null) {
-                ClusterNode n = lookup.get(primaryId);
-
-                assert n != null;
-
-                return Collections.singletonList(n);
-            }
-
-            selected = primaryId != null ? F.concat(false, primaryId, backupIds) : backupIds;
-        }
-        else {
-            if (!exclNeighbors) {
-                selected = nodeHash.nodes(part, backups == Integer.MAX_VALUE ? backups : backups + 1, new P1<NodeInfo>() {
-                    @Override public boolean apply(NodeInfo id) {
-                        return lookup.containsKey(id);
-                    }
-                });
-
-                if (selected.size() == 1) {
-                    NodeInfo id = F.first(selected);
-
-                    assert id != null : "Node ID cannot be null in affinity node ID collection: " + selected;
-
-                    ClusterNode n = lookup.get(id);
-
-                    assert n != null;
-
-                    return Collections.singletonList(n);
-                }
-            }
-            else {
-                int primaryAndBackups = backups + 1;
-
-                selected = new ArrayList<>(primaryAndBackups);
-
-                final Collection<NodeInfo> selected0 = selected;
-
-                List<NodeInfo> ids = nodeHash.nodes(part, primaryAndBackups, new P1<NodeInfo>() {
-                    @Override public boolean apply(NodeInfo id) {
-                        ClusterNode n = lookup.get(id);
-
-                        if (n == null)
-                            return false;
-
-                        Collection<UUID> neighbors = neighbors(n);
-
-                        for (NodeInfo id0 : selected0) {
-                            ClusterNode n0 = lookup.get(id0);
-
-                            if (n0 == null)
-                                return false;
-
-                            Collection<UUID> neighbors0 = neighbors(n0);
-
-                            if (F.containsAny(neighbors0, neighbors))
-                                return false;
-                        }
-
-                        selected0.add(id);
-
-                        return true;
-                    }
-                });
-
-                if (AFFINITY_CONSISTENCY_CHECK)
-                    assert F.eqOrdered(ids, selected);
-            }
-        }
-
-        Collection<ClusterNode> ret = new ArrayList<>(selected.size());
-
-        for (NodeInfo id : selected) {
-            ClusterNode n = lookup.get(id);
-
-            assert n != null;
-
-            ret.add(n);
-        }
-
-        return ret;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int partition(Object key) {
-        initialize();
-
-        return U.safeAbs(key.hashCode() % parts);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int partitions() {
-        initialize();
-
-        return parts;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void reset() {
-        addedNodes = new ConcurrentHashMap<>();
-        neighbors = new ConcurrentHashMap8<>();
-
-        initLatch = new CountDownLatch(1);
-
-        init = new AtomicBoolean();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeNode(UUID nodeId) {
-        NodeInfo info = addedNodes.remove(nodeId);
-
-        if (info == null)
-            return;
-
-        nodeHash.removeNode(info);
-
-        neighbors.clear();
-    }
-
-    /**
-     * Resolve node info for specified node.
-     * Add node to hash circle if this is the first node invocation.
-     *
-     * @param n Node to get info for.
-     * @return Node info.
-     */
-    private NodeInfo resolveNodeInfo(ClusterNode n) {
-        UUID nodeId = n.id();
-        NodeInfo nodeInfo = addedNodes.get(nodeId);
-
-        if (nodeInfo != null)
-            return nodeInfo;
-
-        assert hashIdRslvr != null;
-
-        nodeInfo = new NodeInfo(nodeId, hashIdRslvr.resolve(n), n);
-
-        neighbors.clear();
-
-        nodeHash.addNode(nodeInfo, replicas(n));
-
-        addedNodes.put(nodeId, nodeInfo);
-
-        return nodeInfo;
-    }
-
-    /** {@inheritDoc} */
-    private void initialize() {
-        if (!init.get() && init.compareAndSet(false, true)) {
-            if (log.isInfoEnabled())
-                log.info("Consistent hash configuration [cacheName=" + cacheName + ", partitions=" + parts +
-                    ", excludeNeighbors=" + exclNeighbors + ", replicas=" + replicas +
-                    ", backupFilter=" + backupFilter + ", hashIdRslvr=" + hashIdRslvr + ']');
-
-            nodeHash = new GridConsistentHash<>();
-
-            initLatch.countDown();
-        }
-        else {
-            if (initLatch.getCount() > 0) {
-                try {
-                    U.await(initLatch);
-                }
-                catch (IgniteInterruptedCheckedException ignored) {
-                    // Recover interrupted state flag.
-                    Thread.currentThread().interrupt();
-                }
-            }
-        }
-    }
-
-    /**
-     * @param n Node.
-     * @return Replicas.
-     */
-    private int replicas(ClusterNode n) {
-        Integer nodeReplicas = n.attribute(attrName);
-
-        if (nodeReplicas == null)
-            nodeReplicas = replicas;
-
-        return nodeReplicas;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(CacheConsistentHashAffinityFunction.class, this);
-    }
-
-    /**
-     * Node hash ID.
-     */
-    private static final class NodeInfo implements Comparable<NodeInfo> {
-        /** Node ID. */
-        private UUID nodeId;
-
-        /** Hash ID. */
-        private Object hashId;
-
-        /** Grid node. */
-        private ClusterNode node;
-
-        /**
-         * @param nodeId Node ID.
-         * @param hashId Hash ID.
-         * @param node Rich node.
-         */
-        private NodeInfo(UUID nodeId, Object hashId, ClusterNode node) {
-            assert nodeId != null;
-            assert hashId != null;
-
-            this.hashId = hashId;
-            this.nodeId = nodeId;
-            this.node = node;
-        }
-
-        /**
-         * @return Node ID.
-         */
-        public UUID nodeId() {
-            return nodeId;
-        }
-
-        /**
-         * @return Hash ID.
-         */
-        public Object hashId() {
-            return hashId;
-        }
-
-        /**
-         * @return Node.
-         */
-        public ClusterNode node() {
-            return node;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return hashId.hashCode();
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object obj) {
-            if (!(obj instanceof NodeInfo))
-                return false;
-
-            NodeInfo that = (NodeInfo)obj;
-
-            // If objects are equal, hash codes should be the same.
-            // Cannot use that.hashId.equals(hashId) due to Comparable<N> interface restrictions.
-            return that.nodeId.equals(nodeId) && that.hashCode() == hashCode();
-        }
-
-        /** {@inheritDoc} */
-        @Override public int compareTo(NodeInfo o) {
-            int diff = nodeId.compareTo(o.nodeId);
-
-            if (diff == 0) {
-                int h1 = hashCode();
-                int h2 = o.hashCode();
-
-                diff = h1 == h2 ? 0 : (h1 < h2 ? -1 : 1);
-            }
-
-            return diff;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(NodeInfo.class, this);
-        }
-    }
-}


[49/50] incubator-ignite git commit: IGNITE-45 - WIP

Posted by ag...@apache.org.
IGNITE-45 - WIP


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

Branch: refs/heads/ignite-45
Commit: 30d96ad594b2d9372a400847068e803bd3c9631a
Parents: 55a9c50
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Tue Mar 3 19:21:08 2015 -0800
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Tue Mar 3 19:21:08 2015 -0800

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         | 66 +++++++++++++++++---
 .../affinity/AffinityTopologyVersion.java       | 23 ++++++-
 .../affinity/GridAffinityAssignmentCache.java   |  6 ++
 .../cache/DynamicCacheDescriptor.java           | 28 ++++++---
 .../GridCachePartitionExchangeManager.java      | 54 ++++++++++------
 .../processors/cache/GridCacheProcessor.java    | 36 ++++++++---
 .../cache/GridCacheSharedContext.java           |  7 +++
 .../dht/GridDhtPartitionTopologyImpl.java       |  2 +-
 .../preloader/GridDhtPartitionExchangeId.java   |  5 +-
 .../GridDhtPartitionsExchangeFuture.java        | 34 +++++++++-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  1 +
 .../cache/IgniteDynamicCacheStartSelfTest.java  | 13 ++++
 12 files changed, 219 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/30d96ad5/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 dce04e2..d891149 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
@@ -166,6 +166,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /** Custom event listener. */
     private GridPlainInClosure<Serializable> customEvtLsnr;
 
+    /** Map of dynamic cache filters. */
+    private Map<String, IgnitePredicate<ClusterNode>> dynamicCacheFilters = new HashMap<>();
+
     /** @param ctx Context. */
     public GridDiscoveryManager(GridKernalContext ctx) {
         super(ctx, ctx.config().getDiscoverySpi());
@@ -214,6 +217,18 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         getSpi().setNodeAttributes(attrs, ver);
     }
 
+    /**
+     * Adds dynamic cache filters.
+     *
+     * @param cacheName Cache name.
+     * @param filter Cache filter.
+     */
+    public void addDynamicCacheFilter(String cacheName, IgnitePredicate<ClusterNode> filter) {
+        IgnitePredicate<ClusterNode> old = dynamicCacheFilters.put(cacheName, filter);
+
+        assert old == null;
+    }
+
     /** {@inheritDoc} */
     @Override public void start() throws IgniteCheckedException {
         discoOrdered = discoOrdered();
@@ -277,10 +292,19 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                         c.updateAlives(node);
                 }
 
+                if (type == DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT) {
+                    try {
+                        customEvtLsnr.apply(data);
+                    }
+                    catch (Exception e) {
+                        U.error(log, "Failed to notify direct custom event listener: " + data, e);
+                    }
+                }
+
                 // Put topology snapshot into discovery history.
                 // There is no race possible between history maintenance and concurrent discovery
                 // event notifications, since SPI notifies manager about all events from this listener.
-                if (type != EVT_NODE_METRICS_UPDATED && type != DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT) {
+                if (type != EVT_NODE_METRICS_UPDATED) {
                     DiscoCache cache = new DiscoCache(locNode, F.view(topSnapshot, F.remoteNodes(locNode.id())));
 
                     discoCacheHist.put(topVer, cache);
@@ -307,15 +331,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     return;
                 }
 
-                if (type == DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT) {
-                    try {
-                        customEvtLsnr.apply(data);
-                    }
-                    catch (Exception e) {
-                        U.error(log, "Failed to notify direct custom event listener: " + data, e);
-                    }
-                }
-
                 if (topVer > 0 && (type == EVT_NODE_JOINED || type == EVT_NODE_FAILED || type == EVT_NODE_LEFT)) {
                     boolean set = GridDiscoveryManager.this.topVer.setIfGreater(topVer);
 
@@ -1834,6 +1849,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
                 GridCacheAttributes[] caches = node.attribute(ATTR_CACHE);
 
+                boolean hasCaches = false;
+
                 if (caches != null) {
                     nodesWithCaches.add(node);
 
@@ -1860,6 +1877,35 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                         }
                     }
 
+                    hasCaches = true;
+                }
+
+                for (Map.Entry<String, IgnitePredicate<ClusterNode>> entry : dynamicCacheFilters.entrySet()) {
+                    String cacheName = entry.getKey();
+                    IgnitePredicate<ClusterNode> filter = entry.getValue();
+
+                    if (filter.apply(node)) {
+                        addToMap(cacheMap, cacheName, node);
+
+                        if (alive(node.id()))
+                            addToMap(aliveCacheNodes, maskNull(cacheName), node);
+
+                        addToMap(dhtNodesMap, cacheName, node);
+
+                        // TODO IGNITE-45 client and near caches.
+
+                        if (!loc.id().equals(node.id())) {
+                            addToMap(rmtCacheMap, cacheName, node);
+
+                            if (alive(node.id()))
+                                addToMap(aliveRmtCacheNodes, maskNull(cacheName), node);
+                        }
+
+                        hasCaches = true;
+                    }
+                }
+
+                if (hasCaches) {
                     if (alive(node.id())) {
                         aliveNodesWithCaches.add(node);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/30d96ad5/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
index e276253..cb24ecd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
@@ -89,17 +89,34 @@ public class AffinityTopologyVersion implements Comparable<AffinityTopologyVersi
 
     /** {@inheritDoc} */
     @Override public int compareTo(AffinityTopologyVersion o) {
-        return Long.compare(topVer, o.topVer);
+        int cmp = Long.compare(topVer, o.topVer);
+
+        if (cmp == 0)
+            return Integer.compare(minorTopVer, o.minorTopVer);
+
+        return cmp;
     }
 
     /** {@inheritDoc} */
     @Override public boolean equals(Object o) {
-        return o instanceof AffinityTopologyVersion && topVer == ((AffinityTopologyVersion)o).topVer;
+        if (this == o)
+            return true;
+
+        if (!(o instanceof AffinityTopologyVersion))
+            return false;
+
+        AffinityTopologyVersion that = (AffinityTopologyVersion)o;
+
+        return minorTopVer == that.minorTopVer && topVer == that.topVer;
     }
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        return (int)topVer;
+        int result = (int)(topVer ^ (topVer >>> 32));
+
+        result = 31 * result + minorTopVer;
+
+        return result;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/30d96ad5/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
index b964f83..4bb5885 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
@@ -150,6 +150,8 @@ public class GridAffinityAssignmentCache {
         if (log.isDebugEnabled())
             log.debug("Calculating affinity [topVer=" + topVer + ", locNodeId=" + ctx.localNodeId() +
                 ", discoEvt=" + discoEvt + ']');
+        U.debug(log, "Calculating affinity [topVer=" + topVer + ", locNodeId=" + ctx.localNodeId() +
+            ", discoEvt=" + discoEvt + ']');
 
         GridAffinityAssignment prev = affCache.get(topVer.previous());
 
@@ -162,6 +164,8 @@ public class GridAffinityAssignmentCache {
             // Resolve nodes snapshot for specified topology version.
             Collection<ClusterNode> nodes = ctx.discovery().cacheAffinityNodes(cacheName, topVer.topologyVersion());
 
+            U.debug(log, "Affinity nodes: " + nodes);
+
             sorted = sort(nodes);
         }
 
@@ -187,6 +191,8 @@ public class GridAffinityAssignmentCache {
 
         GridAffinityAssignment updated = new GridAffinityAssignment(topVer, assignment);
 
+        U.debug(log, "Updated assignment: " + updated);
+
         updated = F.addIfAbsent(affCache, topVer, updated);
 
         // Update top version, if required.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/30d96ad5/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 196730c..6a6e227 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
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -28,28 +29,35 @@ import java.io.*;
  * Cache start descriptor.
  */
 public class DynamicCacheDescriptor implements Serializable {
+    /** Cache start ID. */
+    private IgniteUuid startId;
+
     /** Cache configuration. */
     @GridToStringExclude
     private CacheConfiguration cacheCfg;
 
     /** Deploy filter bytes. */
     @GridToStringExclude
-    private byte[] deployFltrBytes;
-
-    /** Cache start ID. */
-    private IgniteUuid startId;
+    private IgnitePredicate<ClusterNode> nodeFilter;
 
     /**
      * @param cacheCfg Cache configuration.
-     * @param deployFltrBytes Deployment filter bytes.
+     * @param nodeFilter Node filter.
      */
-    public DynamicCacheDescriptor(CacheConfiguration cacheCfg, byte[] deployFltrBytes, IgniteUuid startId) {
+    public DynamicCacheDescriptor(CacheConfiguration cacheCfg, IgnitePredicate<ClusterNode> nodeFilter, IgniteUuid startId) {
         this.cacheCfg = cacheCfg;
-        this.deployFltrBytes = deployFltrBytes;
+        this.nodeFilter = nodeFilter;
         this.startId = startId;
     }
 
     /**
+     * @return Start ID.
+     */
+    public IgniteUuid startId() {
+        return startId;
+    }
+
+    /**
      * @return Cache configuration.
      */
     public CacheConfiguration cacheConfiguration() {
@@ -57,10 +65,10 @@ public class DynamicCacheDescriptor implements Serializable {
     }
 
     /**
-     * @return Start ID.
+     * @return Node filter.
      */
-    public IgniteUuid startId() {
-        return startId;
+    public IgnitePredicate<ClusterNode> nodeFilter() {
+        return nodeFilter;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/30d96ad5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 76ecea4..d38161e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.cluster.*;
+import org.apache.ignite.internal.events.*;
 import org.apache.ignite.internal.managers.eventstorage.*;
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
@@ -114,6 +115,9 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
                 final ClusterNode n = e.eventNode();
 
+                GridDhtPartitionExchangeId exchId = null;
+                GridDhtPartitionsExchangeFuture<K, V> exchFut = null;
+
                 if (e.type() != EVT_DISCOVERY_CUSTOM_EVT) {
                     assert !loc.id().equals(n.id());
 
@@ -129,12 +133,30 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                         "Node joined with smaller-than-local " +
                             "order [newOrder=" + n.order() + ", locOrder=" + loc.order() + ']';
 
-                    GridDhtPartitionExchangeId exchId = exchangeId(n.id(),
+                    exchId = exchangeId(n.id(),
                         new AffinityTopologyVersion(e.topologyVersion(), minorTopVer = 0),
                         e.type());
 
-                    GridDhtPartitionsExchangeFuture<K, V> exchFut = exchangeFuture(exchId, e);
+                    exchFut = exchangeFuture(exchId, e, null);
+                }
+                else {
+                    DiscoveryCustomEvent customEvt = (DiscoveryCustomEvent)e;
+
+                    if (customEvt.data() instanceof DynamicCacheDescriptor) {
+                        DynamicCacheDescriptor desc = (DynamicCacheDescriptor)customEvt.data();
+
+                        // Check if this event should trigger partition exchange.
+                        if (cctx.cache().dynamicCacheRegistered(desc)) {
+                            exchId = exchangeId(n.id(),
+                                new AffinityTopologyVersion(e.topologyVersion(), ++minorTopVer),
+                                e.type());
+
+                            exchFut = exchangeFuture(exchId, e, desc);
+                        }
+                    }
+                }
 
+                if (exchId != null) {
                     // Start exchange process.
                     pendingExchangeFuts.add(exchFut);
 
@@ -161,9 +183,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                         }
                     });
                 }
-                else {
-                    // TODO.
-                }
             }
             finally {
                 leaveBusy();
@@ -225,7 +244,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
         assert discoEvt.topologyVersion() == startTopVer.topologyVersion();
 
-        GridDhtPartitionsExchangeFuture<K, V> fut = exchangeFuture(exchId, discoEvt);
+        GridDhtPartitionsExchangeFuture<K, V> fut = exchangeFuture(exchId, discoEvt, null);
 
         new IgniteThread(cctx.gridName(), "exchange-worker", exchWorker).start();
 
@@ -399,16 +418,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     }
 
     /**
-     * Callback to start exchange for dynamically started cache.
-     *
-     * @param cacheDesc Cache descriptor.
-     */
-    public void onCacheDeployed(DynamicCacheDescriptor cacheDesc) {
-        // TODO IGNITE-45 move to exchange future.
-        cctx.kernalContext().cache().onCacheStartFinished(cacheDesc);
-    }
-
-    /**
      * @return {@code True} if topology has changed.
      */
     public boolean topologyChanged() {
@@ -579,11 +588,11 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
      * @return Exchange future.
      */
     GridDhtPartitionsExchangeFuture<K, V> exchangeFuture(GridDhtPartitionExchangeId exchId,
-        @Nullable DiscoveryEvent discoEvt) {
+        @Nullable DiscoveryEvent discoEvt, @Nullable DynamicCacheDescriptor startDesc) {
         GridDhtPartitionsExchangeFuture<K, V> fut;
 
         GridDhtPartitionsExchangeFuture<K, V> old = exchFuts.addx(
-            fut = new GridDhtPartitionsExchangeFuture<>(cctx, busyLock, exchId));
+            fut = new GridDhtPartitionsExchangeFuture<>(cctx, busyLock, exchId, startDesc));
 
         if (old != null)
             fut = old;
@@ -606,6 +615,11 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                     fut.cleanUp();
             }
         }
+
+        DynamicCacheDescriptor desc = exchFut.dynamicCacheDescriptor();
+
+        if (desc != null)
+            cctx.cache().onCacheStartFinished(desc);
     }
 
     /**
@@ -654,7 +668,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                     refreshPartitions();
             }
             else
-                exchangeFuture(msg.exchangeId(), null).onReceive(node.id(), msg);
+                exchangeFuture(msg.exchangeId(), null, null).onReceive(node.id(), msg);
         }
         finally {
             leaveBusy();
@@ -692,7 +706,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                     scheduleResendPartitions();
             }
             else
-                exchangeFuture(msg.exchangeId(), null).onReceive(node.id(), msg);
+                exchangeFuture(msg.exchangeId(), null, null).onReceive(node.id(), msg);
         }
         finally {
             leaveBusy();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/30d96ad5/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 069930e..a22c9a2 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
@@ -1170,6 +1170,30 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param desc Descriptor to check.
+     * @return {@code True} if cache was registered for start and exchange future should be created.
+     */
+    public boolean dynamicCacheRegistered(DynamicCacheDescriptor desc) {
+        return dynamicCaches.get(desc.cacheConfiguration().getName()) == desc;
+    }
+
+    /**
+     * @param startDesc Start descriptor.
+     */
+    public void onCacheStartExchange(DynamicCacheDescriptor startDesc) throws IgniteCheckedException {
+        CacheConfiguration cfg = new CacheConfiguration(startDesc.cacheConfiguration());
+
+        initialize(cfg);
+
+        GridCacheContext cacheCtx = createCache(cfg);
+
+        sharedCtx.addCacheContext(cacheCtx);
+
+        startCache(cacheCtx.cache());
+        onKernalStart(cacheCtx.cache());
+    }
+
+    /**
      * Callback invoked when first exchange future for dynamic cache is completed.
      *
      * @param startDesc Cache start descriptor.
@@ -1256,8 +1280,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         DynamicCacheStartFuture fut = new DynamicCacheStartFuture(ctx, IgniteUuid.fromUuid(ctx.localNodeId()));
 
         try {
-            byte[] filterBytes = ctx.config().getMarshaller().marshal(nodeFilter);
-
             for (CacheConfiguration ccfg0 : ctx.config().getCacheConfiguration()) {
                 if (ccfg0.getName().equals(ccfg.getName()))
                     return new GridFinishedFutureEx<>(new IgniteCheckedException("Failed to start cache " +
@@ -1274,11 +1296,11 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 return new GridFinishedFutureEx<>(new IgniteCheckedException("Failed to start cache " +
                     "(a cache with the same name is already started): " + ccfg.getName()));
 
-            ctx.discovery().sendCustomEvent(new DynamicCacheDescriptor(ccfg, filterBytes, fut.startId()));
+            ctx.discovery().sendCustomEvent(new DynamicCacheDescriptor(ccfg, nodeFilter, fut.startId()));
 
             return fut;
         }
-        catch (IgniteCheckedException e) {
+        catch (Exception e) {
             fut.onDone(e);
 
             // Safety.
@@ -1315,11 +1337,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         DynamicCacheDescriptor old = dynamicCaches.put(ccfg.getName(), startDesc);
 
-        assert old == null : "Dynamic cache map was concurrently modified [new=" + startDesc + ", old=" + old + ']';
-
-        // TODO IGNITE-45 create cache context here.
+        ctx.discovery().addDynamicCacheFilter(ccfg.getName(), startDesc.nodeFilter());
 
-        sharedCtx.exchange().onCacheDeployed(startDesc);
+        assert old == null : "Dynamic cache map was concurrently modified [new=" + startDesc + ", old=" + old + ']';
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/30d96ad5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index e133a17..aadb153 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -115,6 +115,13 @@ public class GridCacheSharedContext<K, V> {
     }
 
     /**
+     * @return Cache processor.
+     */
+    public GridCacheProcessor cache() {
+        return kernalCtx.cache();
+    }
+
+    /**
      * Adds cache context to shared cache context.
      *
      * @param cacheCtx Cache context to add.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/30d96ad5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index 4af7534..e86996d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -209,7 +209,7 @@ class GridDhtPartitionTopologyImpl<K, V> implements GridDhtPartitionTopology<K,
             assert topVer.equals(exchId.topologyVersion()) : "Invalid topology version [topVer=" +
                 topVer + ", exchId=" + exchId + ']';
 
-            if (!exchId.isJoined())
+            if (exchId.isLeft())
                 removeNode(exchId.nodeId());
 
             // In case if node joins, get topology at the time of joining node.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/30d96ad5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
index d101efd..1145bdb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht.preloader;
 
+import org.apache.ignite.internal.events.*;
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -28,6 +29,7 @@ import java.nio.*;
 import java.util.*;
 
 import static org.apache.ignite.events.EventType.*;
+import static org.apache.ignite.internal.events.DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT;
 
 /**
  * Exchange ID.
@@ -54,7 +56,8 @@ public class GridDhtPartitionExchangeId implements Message, Comparable<GridDhtPa
      */
     public GridDhtPartitionExchangeId(UUID nodeId, int evt, @NotNull AffinityTopologyVersion topVer) {
         assert nodeId != null;
-        assert evt == EVT_NODE_LEFT || evt == EVT_NODE_FAILED || evt == EVT_NODE_JOINED;
+        assert evt == EVT_NODE_LEFT || evt == EVT_NODE_FAILED || evt == EVT_NODE_JOINED ||
+            evt == EVT_DISCOVERY_CUSTOM_EVT;
         assert topVer.topologyVersion() > 0;
 
         this.nodeId = nodeId;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/30d96ad5/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 81ab4bf..de87904 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
@@ -142,6 +142,9 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Aff
     /** Logger. */
     private IgniteLogger log;
 
+    /** Dynamic cache start descriptor. */
+    private DynamicCacheDescriptor startDesc;
+
     /**
      * Dummy future created to trigger reassignments if partition
      * topology changed while preloading.
@@ -197,8 +200,12 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Aff
      * @param busyLock Busy lock.
      * @param exchId Exchange ID.
      */
-    public GridDhtPartitionsExchangeFuture(GridCacheSharedContext<K, V> cctx, ReadWriteLock busyLock,
-        GridDhtPartitionExchangeId exchId) {
+    public GridDhtPartitionsExchangeFuture(
+        GridCacheSharedContext<K, V> cctx,
+        ReadWriteLock busyLock,
+        GridDhtPartitionExchangeId exchId,
+        DynamicCacheDescriptor startDesc
+    ) {
         super(cctx.kernalContext());
 
         syncNotify(true);
@@ -213,6 +220,7 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Aff
         this.cctx = cctx;
         this.busyLock = busyLock;
         this.exchId = exchId;
+        this.startDesc = startDesc;
 
         log = cctx.logger(getClass());
 
@@ -379,6 +387,13 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Aff
     }
 
     /**
+     * @return Dynamic cache descriptor.
+     */
+    public DynamicCacheDescriptor dynamicCacheDescriptor() {
+        return startDesc;
+    }
+
+    /**
      * @return Init future.
      */
     IgniteInternalFuture<?> initFuture() {
@@ -422,6 +437,9 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Aff
                 // will return corresponding nodes.
                 U.await(evtLatch);
 
+                if (startDesc != null)
+                    startCache();
+
                 assert discoEvt != null;
 
                 assert exchId.nodeId().equals(discoEvt.eventNode().id());
@@ -433,7 +451,8 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Aff
                 }
 
                 // Grab all alive remote nodes with order of equal or less than last joined node.
-                rmtNodes = new ConcurrentLinkedQueue<>(CU.aliveRemoteCacheNodes(cctx, exchId.topologyVersion().topologyVersion()));
+                rmtNodes = new ConcurrentLinkedQueue<>(CU.aliveRemoteCacheNodes(cctx,
+                    exchId.topologyVersion().topologyVersion()));
 
                 rmtIds = Collections.unmodifiableSet(new HashSet<>(F.nodeIds(rmtNodes)));
 
@@ -547,6 +566,15 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Aff
     }
 
     /**
+     * Starts dynamic cache.
+     */
+    private void startCache() throws IgniteCheckedException {
+        assert startDesc != null;
+
+        ctx.cache().onCacheStartExchange(startDesc);
+    }
+
+    /**
      * @param node Node.
      * @param id ID.
      * @throws IgniteCheckedException If failed.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/30d96ad5/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index 34995ba..81b128e 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -4481,6 +4481,7 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
                 }
 
                 msg.verify(getLocalNodeId());
+                msg.topologyVersion(ring.topologyVersion());
             }
 
             if (msg.verified()) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/30d96ad5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
index 5d515e3..efb5db2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
@@ -136,4 +136,17 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
         assertEquals(1, succeeded);
         assertEquals(threadNum - 1, failed);
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartCacheSimple() throws Exception {
+        final IgniteKernal kernal = (IgniteKernal)grid(0);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setName("TestCacheName3");
+
+        kernal.context().cache().dynamicStartCache(ccfg, F.<ClusterNode>alwaysTrue()).get();
+    }
 }


[43/50] incubator-ignite git commit: # minor cleanup of unused code.

Posted by ag...@apache.org.
# minor cleanup of unused code.


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

Branch: refs/heads/ignite-45
Commit: 2a68e64eb50460b4709a7bc4b8e88be3db0604e8
Parents: d33abdb
Author: AKuznetsov <ak...@gridgain.com>
Authored: Tue Mar 3 16:25:25 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Tue Mar 3 16:25:25 2015 +0700

----------------------------------------------------------------------
 .../visor/cache/VisorCacheAffinityConfiguration.java      | 10 ----------
 .../ignite/visor/commands/cache/VisorCacheCommand.scala   |  1 -
 2 files changed, 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2a68e64e/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
index 6ff2d2d..613b034 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
@@ -46,9 +46,6 @@ public class VisorCacheAffinityConfiguration implements Serializable {
     /** Cache affinity partitions. */
     private Integer partitions;
 
-    /** Cache partitioned affinity default replicas. */
-    private Integer dfltReplicas;
-
     /** Cache partitioned affinity exclude neighbors. */
     private Boolean excludeNeighbors;
 
@@ -107,13 +104,6 @@ public class VisorCacheAffinityConfiguration implements Serializable {
     }
 
     /**
-     * @return Cache partitioned affinity default replicas.
-     */
-    @Nullable public Integer defaultReplicas() {
-        return dfltReplicas;
-    }
-
-    /**
      * @return Cache partitioned affinity exclude neighbors.
      */
     @Nullable public Boolean excludeNeighbors() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2a68e64e/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
index 2db43dc..69da6f5 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
@@ -805,7 +805,6 @@ object VisorCacheCommand {
         cacheT += ("Affinity Function", safe(affinityCfg.function()))
         cacheT += ("Affinity Backups", affinityCfg.partitionedBackups())
         cacheT += ("Affinity Partitions", safe(affinityCfg.partitions()))
-        cacheT += ("Affinity Default Replicas", safe(affinityCfg.defaultReplicas()))
         cacheT += ("Affinity Exclude Neighbors", safe(affinityCfg.excludeNeighbors()))
         cacheT += ("Affinity Mapper", safe(affinityCfg.mapper()))
 


[04/50] incubator-ignite git commit: #ignite-237: change affinity function in tests.

Posted by ag...@apache.org.
#ignite-237: change affinity function in tests.


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

Branch: refs/heads/ignite-45
Commit: d99892590d857c7a9b954cfaff747a0c46b339c8
Parents: f358766
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Feb 26 16:12:03 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Feb 26 16:12:03 2015 +0300

----------------------------------------------------------------------
 ...dCachePartitionedAffinityFilterSelfTest.java |  5 +--
 .../dht/GridCacheDhtEntrySelfTest.java          |  3 +-
 .../dht/GridCacheDhtInternalEntrySelfTest.java  |  3 +-
 .../dht/GridCacheDhtPreloadBigDataSelfTest.java |  3 +-
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |  3 +-
 .../GridCacheDhtPreloadDisabledSelfTest.java    |  3 +-
 .../GridCacheDhtPreloadMessageCountTest.java    |  3 +-
 ...ridCacheDhtPreloadMultiThreadedSelfTest.java |  3 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |  3 +-
 .../GridCacheDhtPreloadStartStopSelfTest.java   |  3 +-
 .../dht/GridCacheDhtPreloadUnloadSelfTest.java  |  3 +-
 .../distributed/dht/GridCacheDhtTestUtils.java  |  6 ++--
 ...ffinityFunctionExcludeNeighborsSelfTest.java | 32 --------------------
 ...AffinityExcludeNeighborsPerformanceTest.java |  3 +-
 ...rtitionedAffinityHashIdResolverSelfTest.java |  4 +--
 .../ignite/testsuites/IgniteCacheTestSuite.java |  1 -
 16 files changed, 29 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9989259/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedAffinityFilterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedAffinityFilterSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedAffinityFilterSelfTest.java
index b78857d..4ccb0be 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedAffinityFilterSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedAffinityFilterSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -67,7 +68,7 @@ public class GridCachePartitionedAffinityFilterSelfTest extends GridCommonAbstra
 
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        CacheConsistentHashAffinityFunction aff = new CacheConsistentHashAffinityFunction();
+        CacheRendezvousAffinityFunction aff = new CacheRendezvousAffinityFunction();
 
         aff.setBackupFilter(backupFilter);
 
@@ -123,7 +124,7 @@ public class GridCachePartitionedAffinityFilterSelfTest extends GridCommonAbstra
      * @throws Exception If failed.
      */
     private void checkPartitions() throws Exception {
-        int partCnt = CacheConsistentHashAffinityFunction.DFLT_PARTITION_COUNT;
+        int partCnt = CacheRendezvousAffinityFunction.DFLT_PARTITION_COUNT;
 
         CacheAffinityFunction aff = cacheConfiguration(grid(0).configuration(), null).getAffinity();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9989259/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
index a1eb4e3..b482165 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
@@ -62,7 +63,7 @@ public class GridCacheDhtEntrySelfTest extends GridCommonAbstractTest {
         CacheConfiguration cacheCfg = defaultCacheConfiguration();
 
         cacheCfg.setCacheMode(PARTITIONED);
-        cacheCfg.setAffinity(new CacheConsistentHashAffinityFunction(false, 10));
+        cacheCfg.setAffinity(new CacheRendezvousAffinityFunction(false, 10));
         cacheCfg.setBackups(0);
         cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
         cacheCfg.setSwapEnabled(false);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9989259/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtInternalEntrySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtInternalEntrySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtInternalEntrySelfTest.java
index cc274ce..1629bcc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtInternalEntrySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtInternalEntrySelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.processors.cache.*;
@@ -65,7 +66,7 @@ public class GridCacheDhtInternalEntrySelfTest extends GridCommonAbstractTest {
 
         cacheCfg.setCacheMode(PARTITIONED);
         cacheCfg.setPreloadMode(SYNC);
-        cacheCfg.setAffinity(new CacheConsistentHashAffinityFunction(false, 2));
+        cacheCfg.setAffinity(new CacheRendezvousAffinityFunction(false, 2));
         cacheCfg.setBackups(0);
         cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
         cacheCfg.setDistributionMode(CacheDistributionMode.NEAR_PARTITIONED);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9989259/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadBigDataSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadBigDataSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadBigDataSelfTest.java
index 36df6bf..ef4ccc9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadBigDataSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadBigDataSelfTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.lifecycle.*;
 import org.apache.ignite.resources.*;
@@ -85,7 +86,7 @@ public class GridCacheDhtPreloadBigDataSelfTest extends GridCommonAbstractTest {
         cc.setPreloadBatchSize(preloadBatchSize);
         cc.setWriteSynchronizationMode(FULL_SYNC);
         cc.setPreloadMode(preloadMode);
-        cc.setAffinity(new CacheConsistentHashAffinityFunction(false, partitions));
+        cc.setAffinity(new CacheRendezvousAffinityFunction(false, partitions));
         cc.setBackups(backups);
 
         TcpDiscoverySpi disco = new TcpDiscoverySpi();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9989259/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java
index fe88081..8ad2ab6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
@@ -75,7 +76,7 @@ public class GridCacheDhtPreloadDelayedSelfTest extends GridCommonAbstractTest {
         cc.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
         cc.setPreloadMode(preloadMode);
         cc.setPreloadPartitionedDelay(delay);
-        cc.setAffinity(new CacheConsistentHashAffinityFunction(false, 128));
+        cc.setAffinity(new CacheRendezvousAffinityFunction(false, 128));
         cc.setBackups(1);
         cc.setAtomicityMode(TRANSACTIONAL);
         cc.setDistributionMode(NEAR_PARTITIONED);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9989259/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
index 1248770..af7d7dc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
@@ -83,7 +84,7 @@ public class GridCacheDhtPreloadDisabledSelfTest extends GridCommonAbstractTest
         cacheCfg.setCacheMode(PARTITIONED);
         cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_ASYNC);
         cacheCfg.setPreloadMode(NONE);
-        cacheCfg.setAffinity(new CacheConsistentHashAffinityFunction(false, partitions));
+        cacheCfg.setAffinity(new CacheRendezvousAffinityFunction(false, partitions));
         cacheCfg.setBackups(backups);
         cacheCfg.setAtomicityMode(TRANSACTIONAL);
         cacheCfg.setDistributionMode(NEAR_PARTITIONED);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9989259/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMessageCountTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMessageCountTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMessageCountTest.java
index e58c16e..8da4d0c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMessageCountTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMessageCountTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.managers.communication.*;
@@ -63,7 +64,7 @@ public class GridCacheDhtPreloadMessageCountTest extends GridCommonAbstractTest
         cc.setCacheMode(PARTITIONED);
         cc.setWriteSynchronizationMode(FULL_SYNC);
         cc.setPreloadMode(preloadMode);
-        cc.setAffinity(new CacheConsistentHashAffinityFunction(false, 521));
+        cc.setAffinity(new CacheRendezvousAffinityFunction(false, 521));
         cc.setBackups(1);
 
         TcpDiscoverySpi disco = new TcpDiscoverySpi();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9989259/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
index 8a8969d..d6b9fa3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -163,7 +164,7 @@ public class GridCacheDhtPreloadMultiThreadedSelfTest extends GridCommonAbstract
 
         for (CacheConfiguration cCfg : cfg.getCacheConfiguration()) {
             if (cCfg.getCacheMode() == CacheMode.PARTITIONED) {
-                cCfg.setAffinity(new CacheConsistentHashAffinityFunction(2048, null));
+                cCfg.setAffinity(new CacheRendezvousAffinityFunction(2048, null));
                 cCfg.setBackups(1);
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9989259/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
index ae94b12..c44b2b8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
@@ -112,7 +113,7 @@ public class GridCacheDhtPreloadSelfTest extends GridCommonAbstractTest {
         cacheCfg.setPreloadBatchSize(preloadBatchSize);
         cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
         cacheCfg.setPreloadMode(preloadMode);
-        cacheCfg.setAffinity(new CacheConsistentHashAffinityFunction(false, partitions));
+        cacheCfg.setAffinity(new CacheRendezvousAffinityFunction(false, partitions));
         cacheCfg.setBackups(backups);
 
         return cacheCfg;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9989259/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java
index 9ff1bf2..aeb7a4a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
@@ -98,7 +99,7 @@ public class GridCacheDhtPreloadStartStopSelfTest extends GridCommonAbstractTest
             cacheCfg.setPreloadBatchSize(preloadBatchSize);
             cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
             cacheCfg.setPreloadMode(preloadMode);
-            cacheCfg.setAffinity(new CacheConsistentHashAffinityFunction(false, partitions));
+            cacheCfg.setAffinity(new CacheRendezvousAffinityFunction(false, partitions));
             cacheCfg.setBackups(backups);
             cacheCfg.setAtomicityMode(TRANSACTIONAL);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9989259/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadUnloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadUnloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadUnloadSelfTest.java
index 480d01c..35b0dfc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadUnloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadUnloadSelfTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.lifecycle.*;
 import org.apache.ignite.resources.*;
@@ -86,7 +87,7 @@ public class GridCacheDhtPreloadUnloadSelfTest extends GridCommonAbstractTest {
         cc.setPreloadBatchSize(preloadBatchSize);
         cc.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
         cc.setPreloadMode(preloadMode);
-        cc.setAffinity(new CacheConsistentHashAffinityFunction(false, partitions));
+        cc.setAffinity(new CacheRendezvousAffinityFunction(false, partitions));
         cc.setBackups(backups);
         cc.setAtomicityMode(TRANSACTIONAL);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9989259/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java
index 0cf6684..94619f2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.*;
@@ -84,13 +85,10 @@ public class GridCacheDhtTestUtils {
      * @param cache Dht cache.
      */
     static void printAffinityInfo(GridCache<?, ?> cache) {
-        CacheConsistentHashAffinityFunction aff =
-            (CacheConsistentHashAffinityFunction)cache.configuration().getAffinity();
-
         System.out.println("Affinity info.");
         System.out.println("----------------------------------");
         System.out.println("Number of key backups: " + cache.configuration().getBackups());
-        System.out.println("Number of cache partitions: " + aff.getPartitions());
+        System.out.println("Number of cache partitions: " + cache.affinity().partitions());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9989259/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheConsistentHashAffinityFunctionExcludeNeighborsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheConsistentHashAffinityFunctionExcludeNeighborsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheConsistentHashAffinityFunctionExcludeNeighborsSelfTest.java
deleted file mode 100644
index af33567..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheConsistentHashAffinityFunctionExcludeNeighborsSelfTest.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.distributed.near;
-
-import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
-
-/**
- * Tests exclude neighbors flag for consistent hash affinity function.
- */
-public class GridCacheConsistentHashAffinityFunctionExcludeNeighborsSelfTest extends
-    GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheAffinityFunction affinityFunction() {
-        return new CacheConsistentHashAffinityFunction(true);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9989259/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinityExcludeNeighborsPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinityExcludeNeighborsPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinityExcludeNeighborsPerformanceTest.java
index cdeaa54..aef1c18 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinityExcludeNeighborsPerformanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinityExcludeNeighborsPerformanceTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.*;
@@ -69,7 +70,7 @@ public class GridCachePartitionedAffinityExcludeNeighborsPerformanceTest extends
 
         cc.setBackups(2);
 
-        CacheAffinityFunction aff = new CacheConsistentHashAffinityFunction(excNeighbores);
+        CacheAffinityFunction aff = new CacheRendezvousAffinityFunction(excNeighbores);
 
         cc.setAffinity(aff);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9989259/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinityHashIdResolverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinityHashIdResolverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinityHashIdResolverSelfTest.java
index ebb89df..a27c486 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinityHashIdResolverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinityHashIdResolverSelfTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.spi.discovery.tcp.*;
@@ -44,7 +44,7 @@ public class GridCachePartitionedAffinityHashIdResolverSelfTest extends GridComm
 
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        CacheConsistentHashAffinityFunction aff = new CacheConsistentHashAffinityFunction();
+        CacheRendezvousAffinityFunction aff = new CacheRendezvousAffinityFunction();
 
         aff.setHashIdResolver(rslvr);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9989259/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 44e4bad..eab626c 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
@@ -174,7 +174,6 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTest(new TestSuite(GridCacheNearReadersSelfTest.class));
         suite.addTest(new TestSuite(GridCacheAtomicNearReadersSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedAffinitySelfTest.class));
-        suite.addTest(new TestSuite(GridCacheConsistentHashAffinityFunctionExcludeNeighborsSelfTest.class));
         suite.addTest(new TestSuite(GridCacheRendezvousAffinityFunctionExcludeNeighborsSelfTest.class));
         suite.addTest(new TestSuite(GridCacheRendezvousAffinityClientSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedProjectionAffinitySelfTest.class));


[26/50] incubator-ignite git commit: #ignite-237: remove tests for consistent hash.

Posted by ag...@apache.org.
#ignite-237: remove tests for consistent hash.


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

Branch: refs/heads/ignite-45
Commit: 5692d7e39a4600b17dd9fafb2003a96a2fb51489
Parents: e378264
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 19:44:08 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 19:44:08 2015 +0300

----------------------------------------------------------------------
 .../GridCachePartitionedAffinitySelfTest.java   | 181 -------------------
 1 file changed, 181 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5692d7e3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinitySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinitySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinitySelfTest.java
index 1559a24..069ffb6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinitySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinitySelfTest.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
@@ -110,186 +109,6 @@ public class GridCachePartitionedAffinitySelfTest extends GridCommonAbstractTest
         return aff.mapKeyToPrimaryAndBackups(key);
     }
 
-    /** Test predefined affinity - must be ported to all clients. */
-    @SuppressWarnings("UnaryPlus")
-    public void testPredefined() throws IgniteCheckedException {
-        CacheRendezvousAffinityFunction aff = new CacheRendezvousAffinityFunction();
-
-        getTestResources().inject(aff);
-
-        aff.setHashIdResolver(new CacheAffinityNodeIdHashResolver());
-
-        List<ClusterNode> nodes = new ArrayList<>();
-
-        nodes.add(createNode("000ea4cd-f449-4dcb-869a-5317c63bd619", 50));
-        nodes.add(createNode("010ea4cd-f449-4dcb-869a-5317c63bd62a", 60));
-        nodes.add(createNode("0209ec54-ff53-4fdb-8239-5a3ac1fb31bd", 70));
-        nodes.add(createNode("0309ec54-ff53-4fdb-8239-5a3ac1fb31ef", 80));
-        nodes.add(createNode("040c9b94-02ae-45a6-9d5c-a066dbdf2636", 90));
-        nodes.add(createNode("050c9b94-02ae-45a6-9d5c-a066dbdf2747", 100));
-        nodes.add(createNode("0601f916-4357-4cfe-a7df-49d4721690bf", 110));
-        nodes.add(createNode("0702f916-4357-4cfe-a7df-49d4721691c0", 120));
-
-        Map<Object, Integer> data = new LinkedHashMap<>();
-
-        data.put("", 4);
-        data.put("asdf", 4);
-        data.put("224ea4cd-f449-4dcb-869a-5317c63bd619", 5);
-        data.put("fdc9ec54-ff53-4fdb-8239-5a3ac1fb31bd", 2);
-        data.put("0f9c9b94-02ae-45a6-9d5c-a066dbdf2636", 2);
-        data.put("d8f1f916-4357-4cfe-a7df-49d4721690bf", 7);
-        data.put("c77ffeae-78a1-4ee6-a0fd-8d197a794412", 3);
-        data.put("35de9f21-3c9b-4f4a-a7d5-3e2c6cb01564", 1);
-        data.put("d67eb652-4e76-47fb-ad4e-cd902d9b868a", 7);
-
-        data.put(0, 4);
-        data.put(1, 7);
-        data.put(12, 5);
-        data.put(123, 6);
-        data.put(1234, 4);
-        data.put(12345, 6);
-        data.put(123456, 6);
-        data.put(1234567, 6);
-        data.put(12345678, 0);
-        data.put(123456789, 7);
-        data.put(1234567890, 7);
-        data.put(1234567890L, 7);
-        data.put(12345678901L, 2);
-        data.put(123456789012L, 1);
-        data.put(1234567890123L, 0);
-        data.put(12345678901234L, 1);
-        data.put(123456789012345L, 6);
-        data.put(1234567890123456L, 7);
-        data.put(-23456789012345L, 4);
-        data.put(-2345678901234L, 1);
-        data.put(-234567890123L, 5);
-        data.put(-23456789012L, 5);
-        data.put(-2345678901L, 7);
-        data.put(-234567890L, 4);
-        data.put(-234567890, 7);
-        data.put(-23456789, 7);
-        data.put(-2345678, 0);
-        data.put(-234567, 6);
-        data.put(-23456, 6);
-        data.put(-2345, 6);
-        data.put(-234, 7);
-        data.put(-23, 5);
-        data.put(-2, 4);
-
-        data.put(0x80000000, 4);
-        data.put(0x7fffffff, 7);
-        data.put(0x8000000000000000L, 4);
-        data.put(0x7fffffffffffffffL, 4);
-
-        data.put(+1.1, 3);
-        data.put(-10.01, 4);
-        data.put(+100.001, 4);
-        data.put(-1000.0001, 4);
-        data.put(+1.7976931348623157E+308, 6);
-        data.put(-1.7976931348623157E+308, 6);
-        data.put(+4.9E-324, 7);
-        data.put(-4.9E-324, 7);
-    }
-
-    /** Test predefined affinity - must be ported to other clients. */
-    @SuppressWarnings("UnaryPlus")
-    public void testPredefinedHashIdResolver() throws IgniteCheckedException {
-        // Use Md5 hasher for this test.
-        CacheRendezvousAffinityFunction aff = new CacheRendezvousAffinityFunction();
-
-        getTestResources().inject(aff);
-
-        aff.setHashIdResolver(new CacheAffinityNodeHashResolver() {
-            @Override public Object resolve(ClusterNode node) {
-                return null;
-            }
-        });
-
-        List<ClusterNode> nodes = new ArrayList<>();
-
-        nodes.add(createNode("000ea4cd-f449-4dcb-869a-5317c63bd619", 50));
-        nodes.add(createNode("010ea4cd-f449-4dcb-869a-5317c63bd62a", 60));
-        nodes.add(createNode("0209ec54-ff53-4fdb-8239-5a3ac1fb31bd", 70));
-        nodes.add(createNode("0309ec54-ff53-4fdb-8239-5a3ac1fb31ef", 80));
-        nodes.add(createNode("040c9b94-02ae-45a6-9d5c-a066dbdf2636", 90));
-        nodes.add(createNode("050c9b94-02ae-45a6-9d5c-a066dbdf2747", 100));
-        nodes.add(createNode("0601f916-4357-4cfe-a7df-49d4721690bf", 110));
-        nodes.add(createNode("0702f916-4357-4cfe-a7df-49d4721691c0", 120));
-
-        Map<Object, Integer> data = new LinkedHashMap<>();
-
-        data.put("", 4);
-        data.put("asdf", 3);
-        data.put("224ea4cd-f449-4dcb-869a-5317c63bd619", 5);
-        data.put("fdc9ec54-ff53-4fdb-8239-5a3ac1fb31bd", 2);
-        data.put("0f9c9b94-02ae-45a6-9d5c-a066dbdf2636", 2);
-        data.put("d8f1f916-4357-4cfe-a7df-49d4721690bf", 4);
-        data.put("c77ffeae-78a1-4ee6-a0fd-8d197a794412", 3);
-        data.put("35de9f21-3c9b-4f4a-a7d5-3e2c6cb01564", 4);
-        data.put("d67eb652-4e76-47fb-ad4e-cd902d9b868a", 2);
-
-        data.put(0, 4);
-        data.put(1, 1);
-        data.put(12, 7);
-        data.put(123, 1);
-        data.put(1234, 6);
-        data.put(12345, 2);
-        data.put(123456, 5);
-        data.put(1234567, 4);
-        data.put(12345678, 6);
-        data.put(123456789, 3);
-        data.put(1234567890, 3);
-        data.put(1234567890L, 3);
-        data.put(12345678901L, 0);
-        data.put(123456789012L, 1);
-        data.put(1234567890123L, 3);
-        data.put(12345678901234L, 5);
-        data.put(123456789012345L, 5);
-        data.put(1234567890123456L, 7);
-        data.put(-23456789012345L, 6);
-        data.put(-2345678901234L, 4);
-        data.put(-234567890123L, 3);
-        data.put(-23456789012L, 0);
-        data.put(-2345678901L, 4);
-        data.put(-234567890L, 5);
-        data.put(-234567890, 3);
-        data.put(-23456789, 3);
-        data.put(-2345678, 6);
-        data.put(-234567, 4);
-        data.put(-23456, 5);
-        data.put(-2345, 2);
-        data.put(-234, 7);
-        data.put(-23, 6);
-        data.put(-2, 6);
-
-        data.put(0x80000000, 7);
-        data.put(0x7fffffff, 1);
-        data.put(0x8000000000000000L, 7);
-        data.put(0x7fffffffffffffffL, 7);
-
-        data.put(+1.1, 2);
-        data.put(-10.01, 0);
-        data.put(+100.001, 2);
-        data.put(-1000.0001, 0);
-        data.put(+1.7976931348623157E+308, 6);
-        data.put(-1.7976931348623157E+308, 1);
-        data.put(+4.9E-324, 1);
-        data.put(-4.9E-324, 1);
-    }
-
-    /**
-     * Create node with specified node id and replica count.
-     *
-     * @param nodeId Node id.
-     * @param replicaCnt Node partitioned affinity replica count.
-     * @return New node with specified node id and replica count.
-     */
-    private ClusterNode createNode(String nodeId, int replicaCnt) {
-        GridTestNode node = new GridTestNode(UUID.fromString(nodeId));
-
-        return node;
-    }
-
     /** @throws Exception If failed. */
     public void testAffinity() throws Exception {
         waitTopologyUpdate();


[27/50] incubator-ignite git commit: #ignite-237: remove unused imports.

Posted by ag...@apache.org.
#ignite-237: remove unused imports.


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

Branch: refs/heads/ignite-45
Commit: 8e587386d284373dbd6ac310dfcb2a9369236867
Parents: 5692d7e
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 19:46:58 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 19:46:58 2015 +0300

----------------------------------------------------------------------
 .../internal/client/impl/ClientPartitionAffinitySelfTest.java    | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e587386/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
index dde345e..7ece534 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
@@ -17,12 +17,8 @@
 
 package org.apache.ignite.internal.client.impl;
 
-import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.client.*;
-import org.apache.ignite.internal.processors.affinity.*;
-import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.junits.common.*;
 


[42/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-237' into sprint-2

Posted by ag...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-237' into sprint-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/d33abdba
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/d33abdba
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/d33abdba

Branch: refs/heads/ignite-45
Commit: d33abdba733987609f592b046d87209e976ba616
Parents: e5e2a8e f144a89
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Mar 3 11:49:54 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Mar 3 11:49:54 2015 +0300

----------------------------------------------------------------------
 .../ClientAbstractMultiThreadedSelfTest.java    |  37 +-
 .../impl/ClientPartitionAffinitySelfTest.java   | 406 -----------
 .../ClientAbstractMultiNodeSelfTest.java        |  54 +-
 .../integration/ClientAbstractSelfTest.java     |   8 -
 .../ClientTcpUnreachableMultiNodeSelfTest.java  |   4 -
 .../client/suite/IgniteClientTestSuite.java     |   1 -
 .../src/test/resources/spring-server-node.xml   |   2 +-
 .../test/resources/spring-server-ssl-node.xml   |   2 +-
 .../java/org/apache/ignite/cache/GridCache.java |   2 +-
 .../CacheConsistentHashAffinityFunction.java    | 702 -------------------
 .../cache/affinity/consistenthash/package.html  |  24 -
 .../CacheRendezvousAffinityFunction.java        |   2 +-
 .../configuration/CacheConfiguration.java       |   2 +-
 .../ignite/internal/client/GridClientNode.java  |   8 -
 .../client/GridClientPartitionAffinity.java     |   2 +-
 .../client/impl/GridClientNodeImpl.java         |  22 -
 .../connection/GridClientNioTcpConnection.java  |   3 +-
 .../processors/cache/GridCacheAttributes.java   |  32 +-
 .../processors/cache/GridCacheProcessor.java    |  29 +-
 .../rest/client/message/GridClientNodeBean.java |  23 -
 .../top/GridTopologyCommandHandler.java         |   8 -
 .../cache/VisorCacheAffinityConfiguration.java  |   9 +-
 .../config/load/dsi-49-server-production.xml    |   2 +-
 .../GridCacheAffinityBackupsSelfTest.java       |   9 -
 .../ignite/IgniteCacheAffinitySelfTest.java     |  30 +-
 ...AffinityProcessorConsistentHashSelfTest.java |  31 -
 ...idCacheConfigurationConsistencySelfTest.java |  46 +-
 ...ridCacheConfigurationValidationSelfTest.java |   8 +-
 .../cache/GridCacheOffHeapSelfTest.java         |  12 +-
 .../GridCachePartitionedAffinitySpreadTest.java |  16 +-
 .../IgniteClientAffinityAssignmentSelfTest.java |  16 +-
 .../GridCacheClientModesAbstractSelfTest.java   |   4 +-
 ...dCachePartitionedAffinityFilterSelfTest.java |   6 +-
 ...GridCachePreloadRestartAbstractSelfTest.java |   4 +-
 .../dht/GridCacheColocatedDebugTest.java        |   4 +-
 .../dht/GridCacheDhtEntrySelfTest.java          |   4 +-
 ...GridCacheDhtEvictionNearReadersSelfTest.java |  13 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |  13 +-
 .../dht/GridCacheDhtInternalEntrySelfTest.java  |   4 +-
 .../dht/GridCacheDhtPreloadBigDataSelfTest.java |   4 +-
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |   4 +-
 .../GridCacheDhtPreloadDisabledSelfTest.java    |   4 +-
 .../GridCacheDhtPreloadMessageCountTest.java    |   4 +-
 ...ridCacheDhtPreloadMultiThreadedSelfTest.java |   4 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |   4 +-
 .../GridCacheDhtPreloadStartStopSelfTest.java   |   4 +-
 .../dht/GridCacheDhtPreloadUnloadSelfTest.java  |   4 +-
 .../distributed/dht/GridCacheDhtTestUtils.java  |   6 +-
 ...dCachePartitionedTopologyChangeSelfTest.java |   7 +-
 ...ridCachePartitionedUnloadEventsSelfTest.java |   5 +-
 ...ffinityFunctionExcludeNeighborsSelfTest.java |  32 -
 ...AffinityExcludeNeighborsPerformanceTest.java |   4 +-
 ...rtitionedAffinityHashIdResolverSelfTest.java |   4 +-
 .../GridCachePartitionedAffinitySelfTest.java   | 226 ------
 .../GridCachePartitionedFullApiSelfTest.java    |  11 +-
 .../GridCachePartitionedNodeRestartTest.java    |   4 +-
 ...ePartitionedOptimisticTxNodeRestartTest.java |   4 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |   4 +-
 ...cheSynchronousEvictionsFailoverSelfTest.java |   4 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |   4 +-
 ...idOffHeapPartitionedMapAbstractSelfTest.java |   4 +-
 ...apPartitionedMapPerformanceAbstractTest.java |   4 +-
 ...GridUnsafePartitionedMapPerformanceTest.java |   4 +-
 .../capacity/spring-capacity-cache.xml          |   2 +-
 .../loadtests/colocation/spring-colocation.xml  |   2 +-
 .../GridCachePartitionedAtomicLongLoadTest.java |   4 +-
 ...apPartitionedMapPerformanceAbstractTest.java |   4 +-
 ...GridUnsafePartitionedMapPerformanceTest.java |   4 +-
 .../ignite/testframework/GridTestUtils.java     |  11 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 -
 .../ignite/testsuites/IgniteCacheTestSuite.java |   1 -
 .../hibernate/HibernateL2CacheSelfTest.java     |   8 +-
 .../processors/cache/GridCacheSwapSelfTest.java |  12 +-
 73 files changed, 160 insertions(+), 1852 deletions(-)
----------------------------------------------------------------------



[17/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237

Posted by ag...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237


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

Branch: refs/heads/ignite-45
Commit: 7fb3890c609f4c04af9b057686dfc3706a50e59d
Parents: 3bab015 4e7463d
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 13:27:20 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 13:27:20 2015 +0300

----------------------------------------------------------------------
 .../cache/VisorCacheMetricsCollectorTask.java   |  10 +-
 .../visor/node/VisorBasicConfiguration.java     |  17 ---
 .../node/VisorNodeEventsCollectorTask.java      |  10 +-
 .../internal/visor/node/VisorNodeGcTask.java    |  10 +-
 .../internal/visor/node/VisorNodePingTask.java  |  10 +-
 .../commands/alert/VisorAlertCommand.scala      |   8 +-
 .../commands/cache/VisorCacheCommand.scala      |  82 +++++++----
 .../config/VisorConfigurationCommand.scala      | 140 ++++++++++---------
 .../commands/disco/VisorDiscoveryCommand.scala  |   2 +-
 .../scala/org/apache/ignite/visor/visor.scala   |  64 ++++++---
 .../commands/tasks/VisorTasksCommandSpec.scala  |   2 +-
 11 files changed, 193 insertions(+), 162 deletions(-)
----------------------------------------------------------------------



[30/50] incubator-ignite git commit: Merge branch 'ignite-237' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-237

Posted by ag...@apache.org.
Merge branch 'ignite-237' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-237


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

Branch: refs/heads/ignite-45
Commit: 6531cbd1d7e6fa937f8536e349c8e8dd9631c17c
Parents: b378307 10ac88a
Author: ivasilinets <va...@gmail.com>
Authored: Sun Mar 1 21:23:31 2015 +0300
Committer: ivasilinets <va...@gmail.com>
Committed: Sun Mar 1 21:23:31 2015 +0300

----------------------------------------------------------------------
 .../ClientAbstractMultiThreadedSelfTest.java    |  38 +-
 .../impl/ClientPartitionAffinitySelfTest.java   |  87 +--
 .../ClientAbstractMultiNodeSelfTest.java        |  53 +-
 .../integration/ClientAbstractSelfTest.java     |   8 -
 .../src/test/resources/spring-server-node.xml   |   2 +-
 .../test/resources/spring-server-ssl-node.xml   |   2 +-
 .../java/org/apache/ignite/cache/GridCache.java |   2 +-
 .../CacheConsistentHashAffinityFunction.java    | 703 -------------------
 .../cache/affinity/consistenthash/package.html  |  24 -
 .../CacheRendezvousAffinityFunction.java        |   2 +-
 .../configuration/CacheConfiguration.java       |   2 +-
 .../processors/cache/GridCacheAttributes.java   |  32 +-
 .../processors/cache/GridCacheProcessor.java    |  21 +-
 .../top/GridTopologyCommandHandler.java         |   8 -
 .../cache/VisorCacheAffinityConfiguration.java  |   9 +-
 .../config/load/dsi-49-server-production.xml    |   2 +-
 .../GridCacheAffinityBackupsSelfTest.java       |   9 -
 .../ignite/IgniteCacheAffinitySelfTest.java     |  30 +-
 ...AffinityProcessorConsistentHashSelfTest.java |  31 -
 ...idCacheConfigurationConsistencySelfTest.java |  46 +-
 ...ridCacheConfigurationValidationSelfTest.java |   8 +-
 .../GridCachePartitionedAffinitySpreadTest.java |  16 +-
 .../IgniteClientAffinityAssignmentSelfTest.java |  16 +-
 .../GridCacheClientModesAbstractSelfTest.java   |   4 +-
 ...dCachePartitionedAffinityFilterSelfTest.java |   6 +-
 ...GridCachePreloadRestartAbstractSelfTest.java |   4 +-
 .../dht/GridCacheColocatedDebugTest.java        |   4 +-
 .../dht/GridCacheDhtEntrySelfTest.java          |   4 +-
 ...GridCacheDhtEvictionNearReadersSelfTest.java |   6 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |  14 +-
 .../dht/GridCacheDhtInternalEntrySelfTest.java  |   4 +-
 .../dht/GridCacheDhtPreloadBigDataSelfTest.java |   4 +-
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |   4 +-
 .../GridCacheDhtPreloadDisabledSelfTest.java    |   4 +-
 .../GridCacheDhtPreloadMessageCountTest.java    |   4 +-
 ...ridCacheDhtPreloadMultiThreadedSelfTest.java |   4 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |   4 +-
 .../GridCacheDhtPreloadStartStopSelfTest.java   |   4 +-
 .../dht/GridCacheDhtPreloadUnloadSelfTest.java  |   4 +-
 .../distributed/dht/GridCacheDhtTestUtils.java  |   6 +-
 ...dCachePartitionedTopologyChangeSelfTest.java |   7 +-
 ...ridCachePartitionedUnloadEventsSelfTest.java |   5 +-
 ...ffinityFunctionExcludeNeighborsSelfTest.java |  32 -
 ...AffinityExcludeNeighborsPerformanceTest.java |   4 +-
 ...rtitionedAffinityHashIdResolverSelfTest.java |   4 +-
 .../GridCachePartitionedAffinitySelfTest.java   | 226 ------
 .../GridCachePartitionedFullApiSelfTest.java    |   6 +-
 .../GridCachePartitionedNodeRestartTest.java    |   4 +-
 ...ePartitionedOptimisticTxNodeRestartTest.java |   4 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |   4 +-
 ...cheSynchronousEvictionsFailoverSelfTest.java |   4 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |  19 +-
 ...idOffHeapPartitionedMapAbstractSelfTest.java |   4 +-
 ...apPartitionedMapPerformanceAbstractTest.java |   4 +-
 ...GridUnsafePartitionedMapPerformanceTest.java |   4 +-
 .../capacity/spring-capacity-cache.xml          |   2 +-
 .../loadtests/colocation/spring-colocation.xml  |   2 +-
 .../GridCachePartitionedAtomicLongLoadTest.java |   4 +-
 ...apPartitionedMapPerformanceAbstractTest.java |   4 +-
 ...GridUnsafePartitionedMapPerformanceTest.java |   4 +-
 .../ignite/testframework/GridTestUtils.java     |  11 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 -
 .../ignite/testsuites/IgniteCacheTestSuite.java |   1 -
 .../hibernate/HibernateL2CacheSelfTest.java     |   8 +-
 64 files changed, 149 insertions(+), 1454 deletions(-)
----------------------------------------------------------------------



[03/50] incubator-ignite git commit: #ignite-237: change affinity function in tests.

Posted by ag...@apache.org.
#ignite-237: change affinity function in tests.


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

Branch: refs/heads/ignite-45
Commit: f358766f944714acb29a9df313db70dd4ff01f8e
Parents: 7d8b54f
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Feb 26 16:09:03 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Feb 26 16:09:03 2015 +0300

----------------------------------------------------------------------
 .../distributed/GridCachePreloadRestartAbstractSelfTest.java  | 4 ++--
 .../dht/GridCachePartitionedTopologyChangeSelfTest.java       | 7 +++----
 .../dht/GridCachePartitionedUnloadEventsSelfTest.java         | 5 ++---
 .../distributed/near/GridCachePartitionedNodeRestartTest.java | 4 ++--
 .../near/GridCachePartitionedOptimisticTxNodeRestartTest.java | 4 ++--
 .../near/GridCachePartitionedTxSalvageSelfTest.java           | 4 ++--
 .../GridCacheSynchronousEvictionsFailoverSelfTest.java        | 4 ++--
 .../offheap/GridOffHeapPartitionedMapAbstractSelfTest.java    | 4 ++--
 .../GridOffHeapPartitionedMapPerformanceAbstractTest.java     | 4 ++--
 .../GridCachePartitionedAtomicLongLoadTest.java               | 4 ++--
 .../GridOffHeapPartitionedMapPerformanceAbstractTest.java     | 4 ++--
 .../unsafe/GridUnsafePartitionedMapPerformanceTest.java       | 4 ++--
 12 files changed, 25 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f358766f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadRestartAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadRestartAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadRestartAbstractSelfTest.java
index da111a6..f3ba5c8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadRestartAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadRestartAbstractSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.spi.discovery.tcp.*;
@@ -114,7 +114,7 @@ public abstract class GridCachePreloadRestartAbstractSelfTest extends GridCommon
         cc.setStartSize(20);
         cc.setPreloadMode(preloadMode);
         cc.setPreloadBatchSize(preloadBatchSize);
-        cc.setAffinity(new CacheConsistentHashAffinityFunction(false, partitions));
+        cc.setAffinity(new CacheRendezvousAffinityFunction(false, partitions));
         cc.setBackups(backups);
         cc.setAtomicityMode(TRANSACTIONAL);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f358766f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java
index c64f9a5..8a2cfa0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
@@ -75,7 +75,7 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
         CacheConfiguration cc = defaultCacheConfiguration();
 
         cc.setCacheMode(PARTITIONED);
-        cc.setAffinity(new CacheConsistentHashAffinityFunction(false, 18));
+        cc.setAffinity(new CacheRendezvousAffinityFunction(false, 18));
         cc.setBackups(1);
         cc.setPreloadMode(SYNC);
         cc.setDistributionMode(PARTITIONED_ONLY);
@@ -498,8 +498,7 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
                 txFut.get(1000);
 
             for (int i = 0; i < 3; i++) {
-                CacheConsistentHashAffinityFunction affinity = (CacheConsistentHashAffinityFunction)((IgniteKernal)grid(i))
-                    .internalCache().context().config().getAffinity();
+                CacheAffinity affinity = grid(i).affinity(null);
 
                 ConcurrentMap addedNodes = U.field(affinity, "addedNodes");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f358766f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedUnloadEventsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedUnloadEventsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedUnloadEventsSelfTest.java
index f8c41f0..f451a34 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedUnloadEventsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedUnloadEventsSelfTest.java
@@ -18,8 +18,7 @@
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
@@ -62,7 +61,7 @@ public class GridCachePartitionedUnloadEventsSelfTest extends GridCommonAbstract
         CacheConfiguration cacheCfg = defaultCacheConfiguration();
         cacheCfg.setCacheMode(PARTITIONED);
         cacheCfg.setPreloadMode(SYNC);
-        cacheCfg.setAffinity(new CacheConsistentHashAffinityFunction(false, 10));
+        cacheCfg.setAffinity(new CacheRendezvousAffinityFunction(false, 10));
         cacheCfg.setBackups(0);
         return cacheCfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f358766f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java
index 30f4a28..40acefb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
-import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 
@@ -45,7 +45,7 @@ public class GridCachePartitionedNodeRestartTest extends GridCacheAbstractNodeRe
         cc.setStartSize(20);
         cc.setPreloadMode(preloadMode);
         cc.setPreloadBatchSize(preloadBatchSize);
-        cc.setAffinity(new CacheConsistentHashAffinityFunction(false, partitions));
+        cc.setAffinity(new CacheRendezvousAffinityFunction(false, partitions));
         cc.setBackups(backups);
 
         c.setCacheConfiguration(cc);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f358766f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java
index e2fd6a5..0d71336 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
-import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.transactions.*;
@@ -44,7 +44,7 @@ public class GridCachePartitionedOptimisticTxNodeRestartTest extends GridCacheAb
         cc.setStartSize(20);
         cc.setPreloadMode(preloadMode);
         cc.setPreloadBatchSize(preloadBatchSize);
-        cc.setAffinity(new CacheConsistentHashAffinityFunction(false, partitions));
+        cc.setAffinity(new CacheRendezvousAffinityFunction(false, partitions));
         cc.setBackups(backups);
 
         c.setCacheConfiguration(cc);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f358766f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
index 685f9d2..b8a3cc9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
@@ -76,7 +76,7 @@ public class GridCachePartitionedTxSalvageSelfTest extends GridCommonAbstractTes
         CacheConfiguration cc = defaultCacheConfiguration();
 
         cc.setCacheMode(CacheMode.PARTITIONED);
-        cc.setAffinity(new CacheConsistentHashAffinityFunction(false, 18));
+        cc.setAffinity(new CacheRendezvousAffinityFunction(false, 18));
         cc.setBackups(1);
         cc.setPreloadMode(CachePreloadMode.SYNC);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f358766f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheSynchronousEvictionsFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheSynchronousEvictionsFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheSynchronousEvictionsFailoverSelfTest.java
index f348d7c..4e32166 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheSynchronousEvictionsFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheSynchronousEvictionsFailoverSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.eviction;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
@@ -64,7 +64,7 @@ public class GridCacheSynchronousEvictionsFailoverSelfTest extends GridCacheAbst
 
         ccfg.setBackups(2);
 
-        ccfg.setAffinity(new CacheConsistentHashAffinityFunction(false, 500));
+        ccfg.setAffinity(new CacheRendezvousAffinityFunction(false, 500));
 
         return ccfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f358766f/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
index 2369749..482b672 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.util.offheap;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.lang.*;
@@ -245,7 +245,7 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm
     public void testPutRandomKeys() throws Exception {
         map = newMap();
 
-        CacheAffinityFunction aff = new CacheConsistentHashAffinityFunction(parts, null);
+        CacheAffinityFunction aff = new CacheRendezvousAffinityFunction(parts, null);
 
         getTestResources().inject(aff);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f358766f/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapPerformanceAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapPerformanceAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapPerformanceAbstractTest.java
index 1f1f806..dd01a66 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapPerformanceAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapPerformanceAbstractTest.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.util.offheap;
 
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -76,7 +76,7 @@ public abstract class GridOffHeapPartitionedMapPerformanceAbstractTest extends G
             keys = new T3[LOAD_CNT];
             wrappers = new GridByteArrayWrapper[LOAD_CNT];
 
-            CacheAffinityFunction aff = new CacheConsistentHashAffinityFunction();
+            CacheAffinityFunction aff = new CacheRendezvousAffinityFunction();
 
             Random rnd = new Random();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f358766f/modules/core/src/test/java/org/apache/ignite/loadtests/datastructures/GridCachePartitionedAtomicLongLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/datastructures/GridCachePartitionedAtomicLongLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/datastructures/GridCachePartitionedAtomicLongLoadTest.java
index 1055102..41abf7c 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/datastructures/GridCachePartitionedAtomicLongLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/datastructures/GridCachePartitionedAtomicLongLoadTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.loadtests.datastructures;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cache.eviction.lru.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.spi.discovery.tcp.*;
@@ -72,7 +72,7 @@ public class GridCachePartitionedAtomicLongLoadTest extends GridCommonAbstractTe
         cc.setWriteSynchronizationMode(FULL_SYNC);
         cc.setEvictionPolicy(new CacheLruEvictionPolicy<>(1000));
         cc.setBackups(1);
-        cc.setAffinity(new CacheConsistentHashAffinityFunction(true));
+        cc.setAffinity(new CacheRendezvousAffinityFunction(true));
         cc.setEvictSynchronized(true);
         cc.setEvictNearSynchronized(true);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f358766f/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapPartitionedMapPerformanceAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapPartitionedMapPerformanceAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapPartitionedMapPerformanceAbstractTest.java
index ab45210..c29c3ca 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapPartitionedMapPerformanceAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapPartitionedMapPerformanceAbstractTest.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.loadtests.offheap;
 
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.offheap.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -77,7 +77,7 @@ public abstract class GridOffHeapPartitionedMapPerformanceAbstractTest extends G
             keys = new T3[LOAD_CNT];
             wrappers = new GridByteArrayWrapper[LOAD_CNT];
 
-            CacheAffinityFunction aff = new CacheConsistentHashAffinityFunction();
+            CacheAffinityFunction aff = new CacheRendezvousAffinityFunction();
 
             Random rnd = new Random();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f358766f/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafePartitionedMapPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafePartitionedMapPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafePartitionedMapPerformanceTest.java
index 5ba7182..a9a7b5b 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafePartitionedMapPerformanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafePartitionedMapPerformanceTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.loadtests.offheap.unsafe;
 
-import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.internal.util.offheap.*;
 
 /**
@@ -26,7 +26,7 @@ import org.apache.ignite.internal.util.offheap.*;
 public class GridUnsafePartitionedMapPerformanceTest extends GridOffHeapPartitionedMapPerformanceAbstractTest {
     /** {@inheritDoc} */
     @Override protected GridOffHeapPartitionedMap newMap() {
-        return GridOffHeapMapFactory.unsafePartitionedMap(CacheConsistentHashAffinityFunction.DFLT_PARTITION_COUNT,
+        return GridOffHeapMapFactory.unsafePartitionedMap(CacheRendezvousAffinityFunction.DFLT_PARTITION_COUNT,
             Runtime.getRuntime().availableProcessors(), load, LOAD_CNT, mem, lruStripes, null);
     }
 }


[19/50] incubator-ignite git commit: #ignite-237: add node attributes.

Posted by ag...@apache.org.
#ignite-237: add node attributes.


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

Branch: refs/heads/ignite-45
Commit: 148ca09572b336cfe4c55683e059ca13fa8cd238
Parents: d254da4
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 15:53:51 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 15:53:51 2015 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/GridCacheAttributes.java       | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/148ca095/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
index 571ae25..8ecd84c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -191,6 +192,13 @@ public class GridCacheAttributes implements Externalizable {
         CacheAffinityFunction aff = cfg.getAffinity();
 
         if (aff != null) {
+            if (aff instanceof CacheRendezvousAffinityFunction) {
+                CacheRendezvousAffinityFunction aff0 = (CacheRendezvousAffinityFunction) aff;
+
+                affInclNeighbors = aff0.isExcludeNeighbors();
+                affHashIdRslvrClsName = className(aff0.getHashIdResolver());
+            }
+
             affPartsCnt = aff.partitions();
             affClsName = className(aff);
         }


[50/50] incubator-ignite git commit: IGNITE-45 - WIP

Posted by ag...@apache.org.
IGNITE-45 - WIP


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

Branch: refs/heads/ignite-45
Commit: 647691f1789ff4b32e21245d03a7f3f6bcd0549a
Parents: 30d96ad
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Tue Mar 3 19:33:01 2015 -0800
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Tue Mar 3 19:33:01 2015 -0800

----------------------------------------------------------------------
 .../cache/distributed/dht/GridDhtPartitionTopologyImpl.java  | 4 ++--
 .../dht/preloader/GridDhtPartitionExchangeId.java            | 8 ++++++++
 2 files changed, 10 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/647691f1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index e86996d..8efd5eb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -248,8 +248,8 @@ class GridDhtPartitionTopologyImpl<K, V> implements GridDhtPartitionTopology<K,
             if (cctx.preloadEnabled()) {
                 for (int p = 0; p < num; p++) {
                     // If this is the first node in grid.
-                    if (oldest.id().equals(loc.id()) && oldest.id().equals(exchId.nodeId())) {
-                        assert exchId.isJoined();
+                    if ((oldest.id().equals(loc.id()) && oldest.id().equals(exchId.nodeId())) || exchId.isCacheAdded()) {
+                        assert exchId.isJoined() || exchId.isCacheAdded();
 
                         try {
                             GridDhtLocalPartition<K, V> locPart = localPartition(p, topVer, true, false);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/647691f1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
index 1145bdb..0326444 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
@@ -107,6 +107,14 @@ public class GridDhtPartitionExchangeId implements Message, Comparable<GridDhtPa
         return evt == EVT_NODE_LEFT || evt == EVT_NODE_FAILED;
     }
 
+    /**
+     * @return {@code True} if cache was added with this exchange ID.
+     */
+    public boolean isCacheAdded() {
+        // TODO IGNITE-45 add cache added flag.
+        return evt == EVT_DISCOVERY_CUSTOM_EVT;
+    }
+
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         U.writeUuid(out, nodeId);


[44/50] incubator-ignite git commit: Make IgniteEvents.withAsync() public (all method in the interfaces must be public)

Posted by ag...@apache.org.
Make IgniteEvents.withAsync() public (all method in the interfaces must be public)


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

Branch: refs/heads/ignite-45
Commit: 6909cc4b45f78dc6ab649462b075bf12a9047eb5
Parents: 2a68e64
Author: sevdokimov <se...@gridgain.com>
Authored: Tue Mar 3 16:36:00 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Tue Mar 3 16:36:00 2015 +0300

----------------------------------------------------------------------
 modules/core/src/main/java/org/apache/ignite/IgniteEvents.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6909cc4b/modules/core/src/main/java/org/apache/ignite/IgniteEvents.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteEvents.java b/modules/core/src/main/java/org/apache/ignite/IgniteEvents.java
index f6adecb..5a92900 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteEvents.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteEvents.java
@@ -257,5 +257,5 @@ public interface IgniteEvents extends IgniteAsyncSupport {
     public boolean isEnabled(int type);
 
     /** {@inheritDoc} */
-    @Override IgniteEvents withAsync();
+    @Override public IgniteEvents withAsync();
 }


[08/50] incubator-ignite git commit: #ignite-237: change affinity function in tests.

Posted by ag...@apache.org.
#ignite-237: change affinity function in tests.


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

Branch: refs/heads/ignite-45
Commit: 96ad6c16cd35a1552d54a515034f1e57edfbffa4
Parents: 81fb145
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Feb 26 16:25:53 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Feb 26 16:25:53 2015 +0300

----------------------------------------------------------------------
 modules/core/src/test/config/load/dsi-49-server-production.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96ad6c16/modules/core/src/test/config/load/dsi-49-server-production.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/load/dsi-49-server-production.xml b/modules/core/src/test/config/load/dsi-49-server-production.xml
index 07fc164..316d889 100644
--- a/modules/core/src/test/config/load/dsi-49-server-production.xml
+++ b/modules/core/src/test/config/load/dsi-49-server-production.xml
@@ -53,7 +53,7 @@
                     <!-- 1 backup -->
                     <property name="backups" value="1"/>
                     <property name="affinity">
-                        <bean class="org.apache.ignite.cache.affinity.consistenthash.CacheConsistentHashAffinityFunction">
+                        <bean class="org.apache.ignite.cache.affinity.rendezvous.CacheRendezvousAffinityFunction">
                             <property name="excludeNeighbors" value="true"/>
                         </bean>
                     </property>


[36/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237

Posted by ag...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237

Conflicts:
	modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.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/9efcd803
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/9efcd803
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/9efcd803

Branch: refs/heads/ignite-45
Commit: 9efcd8035dd74977f44ce7ab6ebd989b57e57482
Parents: 5a76d70 b2675bc
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Mar 2 18:03:07 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Mar 2 18:03:07 2015 +0300

----------------------------------------------------------------------
 examples/pom-standalone.xml                     |   2 +-
 .../java/org/apache/ignite/IgniteCluster.java   |   8 +-
 .../configuration/IgniteConfiguration.java      |   5 +-
 .../ignite/internal/GridKernalContext.java      |  10 +-
 .../ignite/internal/GridKernalContextImpl.java  |  14 +-
 .../apache/ignite/internal/IgniteKernal.java    |  23 +-
 .../org/apache/ignite/internal/IgnitionEx.java  | 592 +++++++++----------
 .../affinity/GridAffinityAssignmentCache.java   |  13 +-
 .../processors/cache/CacheMetricsImpl.java      |   4 +
 .../processors/cache/GridCacheContext.java      |  19 +
 .../processors/cache/GridCacheMapEntry.java     |   9 +
 .../processors/cache/GridCacheProcessor.java    |  15 +
 .../processors/cache/GridCacheStoreManager.java |  12 +-
 .../GridDistributedCacheAdapter.java            |   2 +
 .../distributed/dht/GridDhtCacheEntry.java      |   3 +
 .../distributed/dht/GridDhtLocalPartition.java  |   5 +
 .../dht/atomic/GridDhtAtomicCache.java          |   2 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   2 +-
 .../cache/transactions/IgniteTxManager.java     |   7 +-
 .../processors/cluster/ClusterProcessor.java    |  46 ++
 .../dataload/IgniteDataLoaderImpl.java          |  29 +-
 .../ignite/internal/visor/cache/VisorCache.java | 190 ++----
 .../cache/VisorCacheAffinityConfiguration.java  |  51 +-
 .../visor/cache/VisorCacheConfiguration.java    | 484 +++------------
 .../cache/VisorCacheDefaultConfiguration.java   |  27 +-
 .../cache/VisorCacheEvictionConfiguration.java  |  81 +--
 .../cache/VisorCacheNearConfiguration.java      |  42 +-
 .../cache/VisorCachePreloadConfiguration.java   |  54 +-
 .../cache/VisorCacheStoreConfiguration.java     | 148 ++++-
 .../VisorCacheWriteBehindConfiguration.java     | 137 -----
 .../visor/node/VisorAtomicConfiguration.java    |  27 +-
 .../visor/node/VisorBasicConfiguration.java     | 180 +-----
 .../node/VisorCacheQueryConfiguration.java      |  45 +-
 .../node/VisorExecutorServiceConfiguration.java |  54 +-
 .../visor/node/VisorGridConfiguration.java      | 177 +-----
 .../visor/node/VisorIgfsConfiguration.java      | 244 +-------
 .../visor/node/VisorLifecycleConfiguration.java |   9 +-
 .../visor/node/VisorMetricsConfiguration.java   |  29 +-
 .../node/VisorPeerToPeerConfiguration.java      |  28 +-
 .../visor/node/VisorQueryConfiguration.java     |  65 +-
 .../visor/node/VisorRestConfiguration.java      |  80 +--
 .../node/VisorSegmentationConfiguration.java    |  45 +-
 .../visor/node/VisorSpisConfiguration.java      |  92 +--
 .../node/VisorTransactionConfiguration.java     |  62 +-
 .../internal/visor/util/VisorTaskUtils.java     |   4 +-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../core/src/test/config/store/jdbc/Ignite.xml  |  63 +-
 .../GridCacheAbstractLocalStoreSelfTest.java    | 440 ++++++++++++++
 .../cache/GridCacheAbstractMetricsSelfTest.java |  42 ++
 ...acheAbstractUsersAffinityMapperSelfTest.java | 207 +++++++
 ...dCacheAtomicUsersAffinityMapperSelfTest.java |  45 ++
 .../GridCachePartitionedLocalStoreSelfTest.java |  51 ++
 ...chePartitionedOffHeapLocalStoreSelfTest.java |  56 ++
 .../GridCacheReplicatedLocalStoreSelfTest.java  |  51 ++
 ...heReplicatedUsersAffinityMapperSelfTest.java |  45 ++
 ...ridCacheTxPartitionedLocalStoreSelfTest.java |  51 ++
 .../GridCacheTxUsersAffinityMapperSelfTest.java |  45 ++
 .../GridCacheQueueCleanupSelfTest.java          |   1 -
 .../IgniteCacheExpiryPolicyAbstractTest.java    |  38 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   4 +
 modules/scalar/pom.xml                          |   6 +
 .../ignite/schema/generator/XmlGenerator.java   |   8 +-
 .../apache/ignite/schema/model/PojoField.java   |  11 +-
 .../apache/ignite/schema/load/model/Ignite.xml  | 133 +++--
 .../commands/cache/VisorCacheCommand.scala      |  25 +-
 .../yardstick/config/ignite-store-config.xml    |  15 +-
 pom.xml                                         |   4 +-
 67 files changed, 2105 insertions(+), 2415 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9efcd803/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
index 47a2cf4,0f6a84f..6ff2d2d
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
@@@ -69,10 -71,12 +69,11 @@@ public class VisorCacheAffinityConfigur
  
          VisorCacheAffinityConfiguration cfg = new VisorCacheAffinityConfiguration();
  
-         cfg.function(compactClass(aff));
-         cfg.mapper(compactClass(ccfg.getAffinityMapper()));
-         cfg.partitionedBackups(ccfg.getBackups());
-         cfg.excludeNeighbors(excludeNeighbors);
+         cfg.function = compactClass(aff);
+         cfg.mapper = compactClass(ccfg.getAffinityMapper());
+         cfg.partitions = aff.partitions();
+         cfg.partitionedBackups = ccfg.getBackups();
 -        cfg.dfltReplicas = dfltReplicas;
+         cfg.excludeNeighbors = excludeNeighbors;
  
          return cfg;
      }

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


[14/50] incubator-ignite git commit: #ignite-237: fix ClientAbstractMultiThreadedSelfTest.

Posted by ag...@apache.org.
#ignite-237: fix ClientAbstractMultiThreadedSelfTest.


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

Branch: refs/heads/ignite-45
Commit: 120711b2f8e81634f11b762d3c8f7f0f8aafd18c
Parents: 31180aa
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 12:51:07 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 12:51:07 2015 +0300

----------------------------------------------------------------------
 .../ClientAbstractMultiThreadedSelfTest.java    | 34 +++-----------------
 1 file changed, 5 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/120711b2/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientAbstractMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientAbstractMultiThreadedSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientAbstractMultiThreadedSelfTest.java
index 4ab2679..079b3ec 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientAbstractMultiThreadedSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientAbstractMultiThreadedSelfTest.java
@@ -426,19 +426,6 @@ public abstract class ClientAbstractMultiThreadedSelfTest extends GridCommonAbst
                         String key = String.valueOf(rawKey);
 
                         UUID nodeId = cache.affinity(key);
-                        UUID srvNodeId = ignite.cluster().mapKeyToNode(PARTITIONED_CACHE_NAME, key).id();
-
-                        if (!nodeId.equals(srvNodeId)) {
-                            //GridClientDataAffinity clAff =
-                            //    ((GridClientConfiguration)getFieldValue(client, "cfg")).
-                            //        getDataConfiguration(PARTITIONED_CACHE_NAME).getAffinity();
-
-                            //printAffinityState(gridMap.values());
-                            //info("Client affinity: " + clAff);
-
-                            info("Got wrong client mapping [key=" + key + ", exp=" + srvNodeId +
-                                ", actual=" + nodeId + "]");
-                        }
 
                         String val = "val" + rawKey;
 
@@ -475,25 +462,14 @@ public abstract class ClientAbstractMultiThreadedSelfTest extends GridCommonAbst
                     puts.get(key).get2(), gridMap.get(node.id()).jcache(PARTITIONED_CACHE_NAME).localPeek(key, CachePeekMode.ONHEAP));
             }
 
-            // Assert that client has properly determined affinity node.
-            if (!node.id().equals(puts.get(key).get1())) {
-                //GridClientDataAffinity clAff =
-                //    ((GridClientConfiguration)getFieldValue(client, "cfg")).
-                //        getDataConfiguration(PARTITIONED_CACHE_NAME).getAffinity();
 
-                //printAffinityState(gridMap.values());
-                //info("Client affinity: " + clAff);
-
-                UUID curAffNode = client.data(PARTITIONED_CACHE_NAME).affinity(key);
-
-                failNotEquals(
-                    "Got different mappings [key=" + key + ", currId=" + curAffNode + "]",
-                    node.id(), puts.get(key).get1());
-            }
+            UUID curAffNode = client.data(PARTITIONED_CACHE_NAME).affinity(key);
 
             // Check that no other nodes see this key.
-            for (UUID id : F.view(gridMap.keySet(), F.notEqualTo(node.id())))
-                assertNull("Got value in near cache.", gridMap.get(id).jcache(PARTITIONED_CACHE_NAME).localPeek(key, CachePeekMode.ONHEAP));
+            for (UUID id : gridMap.keySet()) {
+                if (!id.equals(curAffNode) && !id.equals(node.id()))
+                    assertNull("Got value in near cache.", gridMap.get(id).jcache(PARTITIONED_CACHE_NAME).localPeek(key, CachePeekMode.ONHEAP));
+            }
         }
 
         for (Ignite g : gridMap.values())


[12/50] incubator-ignite git commit: #ignite-237: remove unused imports.

Posted by ag...@apache.org.
#ignite-237: remove unused imports.


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

Branch: refs/heads/ignite-45
Commit: eb14f3bbb96865ce656e8facb747e35ff15fbf33
Parents: 75d960e
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Feb 26 18:32:08 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Feb 26 18:32:08 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java   | 1 -
 .../cache/distributed/GridCacheClientModesAbstractSelfTest.java    | 1 -
 .../distributed/GridCachePartitionedAffinityFilterSelfTest.java    | 1 -
 .../cache/distributed/dht/GridCacheColocatedDebugTest.java         | 1 -
 .../cache/distributed/dht/GridCacheDhtEntrySelfTest.java           | 1 -
 .../cache/distributed/dht/GridCacheDhtInternalEntrySelfTest.java   | 1 -
 .../cache/distributed/dht/GridCacheDhtPreloadBigDataSelfTest.java  | 1 -
 .../cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java  | 1 -
 .../cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java | 1 -
 .../cache/distributed/dht/GridCacheDhtPreloadMessageCountTest.java | 1 -
 .../distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java  | 1 -
 .../cache/distributed/dht/GridCacheDhtPreloadSelfTest.java         | 1 -
 .../distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java      | 1 -
 .../cache/distributed/dht/GridCacheDhtPreloadUnloadSelfTest.java   | 1 -
 .../processors/cache/distributed/dht/GridCacheDhtTestUtils.java    | 2 --
 ...ridCachePartitionedAffinityExcludeNeighborsPerformanceTest.java | 1 -
 16 files changed, 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb14f3bb/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java
index 6e34229..88a3cc1 100644
--- a/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java
@@ -19,7 +19,6 @@ package org.apache.ignite;
 
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cache.affinity.fair.*;
 import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb14f3bb/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 170c3d5..a54f73b 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
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache.distributed;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb14f3bb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedAffinityFilterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedAffinityFilterSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedAffinityFilterSelfTest.java
index 4ccb0be..9e0e360 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedAffinityFilterSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedAffinityFilterSelfTest.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb14f3bb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java
index 9cd036f..eb62251 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb14f3bb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
index b482165..91eebea 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb14f3bb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtInternalEntrySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtInternalEntrySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtInternalEntrySelfTest.java
index 1629bcc..504ec12 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtInternalEntrySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtInternalEntrySelfTest.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb14f3bb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadBigDataSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadBigDataSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadBigDataSelfTest.java
index ef4ccc9..587dff1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadBigDataSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadBigDataSelfTest.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.lifecycle.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb14f3bb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java
index 8ad2ab6..33ac600 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb14f3bb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
index af7d7dc..7b9a9bc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb14f3bb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMessageCountTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMessageCountTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMessageCountTest.java
index 8da4d0c..64f01f4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMessageCountTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMessageCountTest.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb14f3bb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
index d6b9fa3..538fb9f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb14f3bb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
index c44b2b8..5c2273c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb14f3bb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java
index aeb7a4a..0f7bcb3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb14f3bb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadUnloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadUnloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadUnloadSelfTest.java
index 35b0dfc..9f6e1a4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadUnloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadUnloadSelfTest.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.lifecycle.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb14f3bb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java
index 94619f2..e9b8a23 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java
@@ -20,8 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
-import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb14f3bb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinityExcludeNeighborsPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinityExcludeNeighborsPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinityExcludeNeighborsPerformanceTest.java
index aef1c18..dfd9128 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinityExcludeNeighborsPerformanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinityExcludeNeighborsPerformanceTest.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;


[32/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237

Posted by ag...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237


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

Branch: refs/heads/ignite-45
Commit: 6c67bdf27f962aee64414c90965e8a86a1cc907d
Parents: 6deba53 c9f46c1
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Mar 2 11:45:01 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Mar 2 11:45:01 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/IgniteUtils.java       |  90 ++++++++++--
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 137 +++++++++++--------
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   | 116 ++++++++++++++++
 3 files changed, 275 insertions(+), 68 deletions(-)
----------------------------------------------------------------------



[28/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237

Posted by ag...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237


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

Branch: refs/heads/ignite-45
Commit: 10ac88a701618249f4e23ecb467fe0099c034a4e
Parents: 8e58738 a792c99
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 20:03:24 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 20:03:24 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/GridProperties.java  | 78 -------------------
 .../ignite/internal/GridUpdateNotifier.java     |  2 +-
 .../apache/ignite/internal/IgniteKernal.java    | 18 ++---
 .../ignite/internal/IgniteProperties.java       | 79 ++++++++++++++++++++
 .../ignite/internal/IgniteVersionUtils.java     |  8 +-
 .../plugin/IgnitePluginProcessor.java           | 24 ++++++
 .../apache/ignite/plugin/PluginProvider.java    |  5 ++
 .../internal/GridUpdateNotifierSelfTest.java    |  2 +-
 8 files changed, 123 insertions(+), 93 deletions(-)
----------------------------------------------------------------------



[16/50] incubator-ignite git commit: #ignite-237: fix ClientAbstractMultiNodeSelfTest.

Posted by ag...@apache.org.
#ignite-237: fix ClientAbstractMultiNodeSelfTest.


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

Branch: refs/heads/ignite-45
Commit: 3bab015f3e7e086972b90998832d6003d05b7c1a
Parents: 4d54c50
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 13:01:08 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 13:01:08 2015 +0300

----------------------------------------------------------------------
 .../ClientAbstractMultiNodeSelfTest.java        | 49 +-------------------
 1 file changed, 1 insertion(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3bab015f/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java
index 0eb6e16..e73f06d 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java
@@ -408,47 +408,6 @@ public abstract class ClientAbstractMultiNodeSelfTest extends GridCommonAbstract
         }
     }
 
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClientAffinity() throws Exception {
-        GridClientData partitioned = client.data(PARTITIONED_CACHE_NAME);
-
-        Collection<Object> keys = new ArrayList<>();
-
-        keys.addAll(Arrays.asList(
-            Boolean.TRUE,
-            Boolean.FALSE,
-            1,
-            Integer.MAX_VALUE
-        ));
-
-        Random rnd = new Random();
-        StringBuilder sb = new StringBuilder();
-
-        // Generate some random strings.
-        for (int i = 0; i < 100; i++) {
-            sb.setLength(0);
-
-            for (int j = 0; j < 255; j++)
-                // Only printable ASCII symbols for test.
-                sb.append((char)(rnd.nextInt(0x7f - 0x20) + 0x20));
-
-            keys.add(sb.toString());
-        }
-
-        // Generate some more keys to achieve better coverage.
-        for (int i = 0; i < 100; i++)
-            keys.add(UUID.randomUUID());
-
-        for (Object key : keys) {
-            UUID nodeId = grid(0).cluster().mapKeyToNode(PARTITIONED_CACHE_NAME, key).id();
-
-            UUID clientNodeId = partitioned.affinity(key);
-
-            assertEquals("Invalid affinity mapping for REST response for key: " + key, nodeId, clientNodeId);
-        }
-    }
 
     /**
      * @throws Exception If failed.
@@ -585,18 +544,12 @@ public abstract class ClientAbstractMultiNodeSelfTest extends GridCommonAbstract
 
             UUID primaryNodeId = grid(0).cluster().mapKeyToNode(PARTITIONED_CACHE_NAME, key).id();
 
-            assertEquals("Affinity mismatch for key: " + key, primaryNodeId, partitioned.affinity(key));
-
-            assertEquals(primaryNodeId, partitioned.affinity(key));
-
-            // Must go to primary node only. Since backup count is 0, value must present on
-            // primary node only.
             partitioned.put(key, "val" + key);
 
             for (Map.Entry<UUID, Ignite> entry : gridsByLocNode.entrySet()) {
                 Object val = entry.getValue().jcache(PARTITIONED_CACHE_NAME).localPeek(key, CachePeekMode.ONHEAP);
 
-                if (primaryNodeId.equals(entry.getKey()))
+                if (primaryNodeId.equals(entry.getKey()) || partitioned.affinity(key).equals(entry.getKey()))
                     assertEquals("val" + key, val);
                 else
                     assertNull(val);


[31/50] incubator-ignite git commit: #ignite-237: remove debug.

Posted by ag...@apache.org.
#ignite-237: remove debug.


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

Branch: refs/heads/ignite-45
Commit: 6deba53c2747432c36503ba83932199a943cdcb7
Parents: 6531cbd
Author: ivasilinets <va...@gmail.com>
Authored: Sun Mar 1 22:59:47 2015 +0300
Committer: ivasilinets <va...@gmail.com>
Committed: Sun Mar 1 22:59:47 2015 +0300

----------------------------------------------------------------------
 ...GridCacheContinuousQueryAbstractSelfTest.java | 19 +++----------------
 1 file changed, 3 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6deba53c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
index 850a213..db1ac5a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
@@ -159,23 +159,10 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
             }
         }
 
-        boolean allEmpty = true;
-
-        for (int i = 0; i < gridCount(); i++) {
-            Set<Cache.Entry<Object, Object>> entries = ((IgniteKernal) grid(i)).cache(null).entrySet();
-
-            for (Cache.Entry entry : entries) {
-                boolean primary = grid(i).affinity(null).isPrimary(grid(i).localNode(), entry.getKey());
-
-                boolean backup = grid(i).affinity(null).isBackup(grid(i).localNode(), entry.getKey());
-
-                info("Not removed entry [grid=" + i + ", primary=" + primary + ", backup=" + backup + ']');
-
-                allEmpty = false;
-            }
-        }
+        for (int i = 0; i < gridCount(); i++)
+            assertEquals("Cache is not empty: " + ((IgniteKernal)grid(i)).cache(null).entrySet(), 0,
+                ((IgniteKernal)grid(i)).cache(null).size());
 
-        assertTrue(allEmpty);
 
         for (int i = 0; i < gridCount(); i++) {
             GridContinuousProcessor proc = ((IgniteKernal)grid(i)).context().continuous();


[35/50] incubator-ignite git commit: #ignite-334: Change IgniteConfiguration.pluginConfigurations to array.

Posted by ag...@apache.org.
#ignite-334: Change IgniteConfiguration.pluginConfigurations to array.


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

Branch: refs/heads/ignite-45
Commit: 4df7c85990391206c7e6247658267185f11f86a9
Parents: c847e88
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Mar 2 13:44:40 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Mar 2 13:44:40 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/configuration/IgniteConfiguration.java   | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4df7c859/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index 72ab928..54404c5 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -311,7 +311,7 @@ public class IgniteConfiguration {
     private TransactionConfiguration txCfg = new TransactionConfiguration();
 
     /** */
-    private Collection<? extends PluginConfiguration> pluginCfgs;
+    private PluginConfiguration[] pluginCfgs;
 
     /** Flag indicating whether cache sanity check is enabled. */
     private boolean cacheSanityCheckEnabled = DFLT_CACHE_SANITY_CHECK_ENABLED;
@@ -1874,14 +1874,14 @@ public class IgniteConfiguration {
     /**
      * @return Plugin configurations.
      */
-    public Collection<? extends PluginConfiguration> getPluginConfigurations() {
+    public PluginConfiguration[] getPluginConfigurations() {
         return pluginCfgs;
     }
 
     /**
      * @param pluginCfgs Plugin configurations.
      */
-    public void setPluginConfigurations(Collection<? extends PluginConfiguration> pluginCfgs) {
+    public void setPluginConfigurations(PluginConfiguration... pluginCfgs) {
         this.pluginCfgs = pluginCfgs;
     }
 


[22/50] incubator-ignite git commit: Merge branches 'ignite-237' and 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-237

Posted by ag...@apache.org.
Merge branches 'ignite-237' and 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-237

Conflicts:
	modules/core/src/main/java/org/apache/ignite/marshaller/optimized/optimized-classnames.properties


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

Branch: refs/heads/ignite-45
Commit: 0d70de861e9f9beb8bc237ce81909efccd18866c
Parents: e50d1ea f053746
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 18:28:17 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 18:28:17 2015 +0300

----------------------------------------------------------------------
 config/ignite-log4j.xml                         |    2 +-
 .../datagrid/CacheContinuousQueryExample.java   |    2 +-
 ipc/shmem/Makefile.am                           |   15 +
 ipc/shmem/igniteshmem/Makefile.am               |   15 +
 ipc/shmem/include/Makefile.am                   |   15 +
 modules/clients/src/test/keystore/generate.sh   |   15 +-
 .../ignite/cache/query/ContinuousQuery.java     |   18 +-
 .../apache/ignite/cluster/ClusterMetrics.java   |    2 +-
 .../internal/events/DiscoveryCustomEvent.java   |    3 +
 .../processors/cache/IgniteCacheProxy.java      |    6 +-
 .../optimized-classnames.previous.properties    |   15 +
 .../optimized/optimized-classnames.properties   | 1565 +-----------------
 .../TcpDiscoveryCustomEventMessage.java         |    3 +
 ...ridCacheContinuousQueryAbstractSelfTest.java |    8 +-
 modules/extdata/p2p/pom.xml                     |    6 -
 modules/hibernate/pom.xml                       |    6 -
 .../HibernateReadWriteAccessStrategy.java       |   81 +-
 modules/indexing/pom.xml                        |    6 -
 modules/jta/pom.xml                             |    6 -
 modules/scalar/pom.xml                          |    6 -
 modules/spring/pom.xml                          |    6 -
 modules/visor-console/pom.xml                   |    7 -
 modules/web/pom.xml                             |    6 -
 modules/winservice/IgniteService.sln            |    2 +-
 .../IgniteService/IgniteService.csproj          |    2 +-
 .../config/benchmark-atomic-win.properties      |   15 +
 .../config/benchmark-atomic.properties          |   15 +
 .../config/benchmark-compute-win.properties     |   15 +
 .../config/benchmark-compute.properties         |   15 +
 .../config/benchmark-multicast.properties       |   15 +
 .../config/benchmark-query-win.properties       |   15 +
 .../yardstick/config/benchmark-query.properties |   15 +
 .../config/benchmark-tx-win.properties          |   15 +
 .../yardstick/config/benchmark-tx.properties    |   15 +
 .../yardstick/config/benchmark-win.properties   |   15 +
 modules/yardstick/config/benchmark.properties   |   15 +
 pom.xml                                         |  150 +-
 37 files changed, 421 insertions(+), 1712 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0d70de86/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
----------------------------------------------------------------------


[39/50] incubator-ignite git commit: # IGNITE-380 Added snippet generation.

Posted by ag...@apache.org.
# IGNITE-380 Added snippet generation.


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

Branch: refs/heads/ignite-45
Commit: 7488eddd6e0224d18ecdd9b7fb51566e926ead5d
Parents: b2675bc
Author: AKuznetsov <ak...@gridgain.com>
Authored: Tue Mar 3 14:49:27 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Tue Mar 3 14:49:27 2015 +0700

----------------------------------------------------------------------
 .../ignite/cache/CacheTypeFieldMetadata.java    |   2 +-
 .../apache/ignite/cache/CacheTypeMetadata.java  |  18 +--
 .../ignite/schema/generator/PojoGenerator.java  |   3 +-
 .../schema/generator/SnippetGenerator.java      | 138 +++++++++++++++++++
 .../apache/ignite/schema/ui/SchemaLoadApp.java  |   2 +
 5 files changed, 151 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7488eddd/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeFieldMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeFieldMetadata.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeFieldMetadata.java
index 0041f10..625873b 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeFieldMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeFieldMetadata.java
@@ -50,7 +50,7 @@ public class CacheTypeFieldMetadata {
      * @param javaName Field name in java object.
      * @param javaType Field java type.
      */
-    public CacheTypeFieldMetadata(String javaName, Class<?> javaType, String dbName, int dbType) {
+    public CacheTypeFieldMetadata(String dbName, int dbType, String javaName, Class<?> javaType) {
         this.dbName = dbName;
         this.dbType = dbType;
         this.javaName = javaName;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7488eddd/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
index 1e502c9..f25d9f4 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
@@ -230,6 +230,15 @@ public class CacheTypeMetadata {
     }
 
     /**
+     * Sets value fields.
+     *
+     * @param valFields New value fields.
+     */
+    public void setValueFields(Collection<CacheTypeFieldMetadata> valFields) {
+        this.valFields = valFields;
+    }
+
+    /**
      * Gets query-enabled fields.
      *
      * @return Collection of fields available for query.
@@ -318,13 +327,4 @@ public class CacheTypeMetadata {
     public void setGroups(Map<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> grps) {
         this.grps = grps;
     }
-
-    /**
-     * Sets value fields.
-     *
-     * @param valFields New value fields.
-     */
-    public void setValueFields(Collection<CacheTypeFieldMetadata> valFields) {
-        this.valFields = valFields;
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7488eddd/modules/schema-load/src/main/java/org/apache/ignite/schema/generator/PojoGenerator.java
----------------------------------------------------------------------
diff --git a/modules/schema-load/src/main/java/org/apache/ignite/schema/generator/PojoGenerator.java b/modules/schema-load/src/main/java/org/apache/ignite/schema/generator/PojoGenerator.java
index 7d12e2c..500aa9a 100644
--- a/modules/schema-load/src/main/java/org/apache/ignite/schema/generator/PojoGenerator.java
+++ b/modules/schema-load/src/main/java/org/apache/ignite/schema/generator/PojoGenerator.java
@@ -401,8 +401,7 @@ public class PojoGenerator {
      * @throws IOException If failed to write generated code into file.
      */
     public static void generate(PojoDescriptor pojo, String outFolder, String pkg, boolean constructor,
-        boolean includeKeys, ConfirmCallable askOverwrite)
-        throws IOException {
+        boolean includeKeys, ConfirmCallable askOverwrite) throws IOException {
         File pkgFolder = new File(outFolder, pkg.replace('.', File.separatorChar));
 
         if (!pkgFolder.exists() && !pkgFolder.mkdirs())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7488eddd/modules/schema-load/src/main/java/org/apache/ignite/schema/generator/SnippetGenerator.java
----------------------------------------------------------------------
diff --git a/modules/schema-load/src/main/java/org/apache/ignite/schema/generator/SnippetGenerator.java b/modules/schema-load/src/main/java/org/apache/ignite/schema/generator/SnippetGenerator.java
new file mode 100644
index 0000000..e54c019
--- /dev/null
+++ b/modules/schema-load/src/main/java/org/apache/ignite/schema/generator/SnippetGenerator.java
@@ -0,0 +1,138 @@
+/*
+ * 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.schema.generator;
+
+import org.apache.ignite.schema.model.*;
+import org.apache.ignite.schema.ui.*;
+
+import java.io.*;
+import java.util.*;
+
+import static org.apache.ignite.schema.ui.MessageBox.Result.*;
+
+/**
+ * Cache configuration snippet generator.
+ */
+public class SnippetGenerator {
+    /**
+     * Add type fields.
+     *
+     * @param src Source code lines.
+     * @param owner Fields owner collection.
+     * @param fields Fields metadata.
+     */
+    private static void addFields(Collection<String> src, String owner, Collection<PojoField> fields) {
+        for (PojoField field : fields) {
+            String javaTypeName = field.javaTypeName();
+
+            if (javaTypeName.startsWith("java.lang."))
+                javaTypeName = javaTypeName.substring(10);
+
+            src.add(owner + ".add(new CacheTypeFieldMetadata(\"" + field.dbName() + "\", " +
+                "java.sql.Types." + field.dbTypeName() + ",\"" +
+                field.javaName() + "\", " + javaTypeName + ".class));");
+        }
+    }
+
+    /**
+     * Generate java snippet for cache configuration with JDBC store.
+     *
+     * @param pojos POJO descriptors.
+     * @param pkg Types package.
+     * @param includeKeys {@code true} if key fields should be included into value class.
+     * @param out File to output snippet.
+     * @param askOverwrite Callback to ask user to confirm file overwrite.
+     * @throws IOException If generation failed.
+     */
+    public static void generate(Collection<PojoDescriptor> pojos, String pkg, boolean includeKeys, File out,
+        ConfirmCallable askOverwrite) throws IOException {
+        if (out.exists()) {
+            MessageBox.Result choice = askOverwrite.confirm(out.getName());
+
+            if (CANCEL == choice)
+                throw new IllegalStateException("Java configuration snippet generation was canceled!");
+
+            if (NO == choice || NO_TO_ALL == choice)
+                return;
+        }
+
+        Collection<String> src = new ArrayList<>(256);
+
+        src.add("// Code snippet for cache configuration.");
+        src.add("");
+        src.add("IgniteConfiguration cfg = new IgniteConfiguration();");
+        src.add("");
+        src.add("CacheConfiguration ccfg = new CacheConfiguration<>();");
+        src.add("");
+        src.add("DataSource dataSource = null; // TODO: Create data source for your database.");
+        src.add("");
+        src.add("// Create store. ");
+        src.add("CacheJdbcPojoStore store = new CacheJdbcPojoStore();");
+        src.add("store.setDataSource(dataSource);");
+        src.add("");
+        src.add("// Create store factory. ");
+        src.add("ccfg.setCacheStoreFactory(new FactoryBuilder.SingletonFactory<>(store));");
+        src.add("");
+        src.add("// Configure cache to use store. ");
+        src.add("ccfg.setReadThrough(true);");
+        src.add("ccfg.setWriteThrough(true);");
+        src.add("");
+        src.add("cfg.setCacheConfiguration(ccfg);");
+        src.add("");
+        src.add("// Configure cache types. ");
+        src.add("Collection<CacheTypeMetadata> meta = new ArrayList<>();");
+        src.add("");
+
+        boolean first = true;
+
+        for (PojoDescriptor pojo : pojos) {
+            String tbl = pojo.table();
+
+            src.add("// " + tbl + ".");
+            src.add((first ? "CacheTypeMetadata " : "") +  "type = new CacheTypeMetadata();");
+            src.add("type.setDatabaseSchema(\"" + pojo.schema() + "\");");
+            src.add("type.setDatabaseTable(\"" + tbl + "\");");
+            src.add("type.setKeyType(\"" + pkg + "." + pojo.keyClassName() + "\");");
+            src.add("type.setValueType(\"" +  pkg + "." + pojo.valueClassName() + "\");");
+            src.add("");
+
+            src.add("// Key fields for " + tbl + ".");
+            src.add((first ? "Collection<CacheTypeFieldMetadata> " : "") + "keys = new ArrayList<>();");
+            addFields(src, "keys", pojo.valueFields(includeKeys));
+            src.add("type.setKeyFields(keys);");
+            src.add("");
+
+            src.add("// Value fields for " + tbl + ".");
+            src.add((first ? "Collection<CacheTypeFieldMetadata> " : "") + "vals = new ArrayList<>();");
+            addFields(src, "vals", pojo.valueFields(includeKeys));
+            src.add("type.setValueFields(vals);");
+            src.add("");
+
+            first = false;
+        }
+
+        src.add("// Start Ignite node.");
+        src.add("Ignition.start(cfg);");
+
+        // Write generated code to file.
+        try (Writer writer = new BufferedWriter(new FileWriter(out))) {
+            for (String line : src)
+                writer.write(line + '\n');
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7488eddd/modules/schema-load/src/main/java/org/apache/ignite/schema/ui/SchemaLoadApp.java
----------------------------------------------------------------------
diff --git a/modules/schema-load/src/main/java/org/apache/ignite/schema/ui/SchemaLoadApp.java b/modules/schema-load/src/main/java/org/apache/ignite/schema/ui/SchemaLoadApp.java
index 9c88624..f43c5ca 100644
--- a/modules/schema-load/src/main/java/org/apache/ignite/schema/ui/SchemaLoadApp.java
+++ b/modules/schema-load/src/main/java/org/apache/ignite/schema/ui/SchemaLoadApp.java
@@ -435,6 +435,8 @@ public class SchemaLoadApp extends Application {
                 if (singleXml)
                     XmlGenerator.generate(pkg, all, includeKeys, new File(outFolder, "Ignite.xml"), askOverwrite);
 
+                SnippetGenerator.generate(all, pkg, includeKeys, new File(outFolder, "Ignite.snippet"), askOverwrite);
+
                 perceptualDelay(started);
 
                 return null;


[02/50] incubator-ignite git commit: #ignite-237: change affinity function in GridUnsafePartitionedMapPerformanceTest.

Posted by ag...@apache.org.
#ignite-237: change affinity function in GridUnsafePartitionedMapPerformanceTest.


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

Branch: refs/heads/ignite-45
Commit: 7d8b54f868fa34d9dbc82c46e759a5a9b32a01e0
Parents: df3c76e
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Feb 26 15:55:07 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Feb 26 15:55:07 2015 +0300

----------------------------------------------------------------------
 .../offheap/unsafe/GridUnsafePartitionedMapPerformanceTest.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7d8b54f8/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafePartitionedMapPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafePartitionedMapPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafePartitionedMapPerformanceTest.java
index 5760b79..2385526 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafePartitionedMapPerformanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafePartitionedMapPerformanceTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.util.offheap.unsafe;
 
-import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.internal.util.offheap.*;
 
 /**
@@ -26,7 +26,7 @@ import org.apache.ignite.internal.util.offheap.*;
 public class GridUnsafePartitionedMapPerformanceTest extends GridOffHeapPartitionedMapPerformanceAbstractTest {
     /** {@inheritDoc} */
     @Override protected GridOffHeapPartitionedMap newMap() {
-        return GridOffHeapMapFactory.unsafePartitionedMap(CacheConsistentHashAffinityFunction.DFLT_PARTITION_COUNT,
+        return GridOffHeapMapFactory.unsafePartitionedMap(CacheRendezvousAffinityFunction.DFLT_PARTITION_COUNT,
             Runtime.getRuntime().availableProcessors(), load, LOAD_CNT, mem, lruStripes, null);
     }
 }