You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/01/16 16:43:13 UTC

[10/12] incubator-ignite git commit: #IGNITE-99: Refactoring. Move GridCache.affinity() to Ignite.affinity(String cacheName).

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/25609651/modules/core/src/main/java/org/apache/ignite/fs/IgniteFsGroupDataBlocksKeyMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/fs/IgniteFsGroupDataBlocksKeyMapper.java b/modules/core/src/main/java/org/apache/ignite/fs/IgniteFsGroupDataBlocksKeyMapper.java
index f07f53c..fcf01d7 100644
--- a/modules/core/src/main/java/org/apache/ignite/fs/IgniteFsGroupDataBlocksKeyMapper.java
+++ b/modules/core/src/main/java/org/apache/ignite/fs/IgniteFsGroupDataBlocksKeyMapper.java
@@ -53,7 +53,7 @@ import org.gridgain.grid.util.typedef.internal.*;
  * </bean>
  * </pre>
  */
-public class IgniteFsGroupDataBlocksKeyMapper extends GridCacheDefaultAffinityKeyMapper {
+public class IgniteFsGroupDataBlocksKeyMapper extends CacheDefaultAffinityKeyMapper {
     /** */
     private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/25609651/modules/core/src/main/java/org/apache/ignite/streamer/router/StreamerCacheAffinityEventRouter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/streamer/router/StreamerCacheAffinityEventRouter.java b/modules/core/src/main/java/org/apache/ignite/streamer/router/StreamerCacheAffinityEventRouter.java
index b6de55d..b258f48 100644
--- a/modules/core/src/main/java/org/apache/ignite/streamer/router/StreamerCacheAffinityEventRouter.java
+++ b/modules/core/src/main/java/org/apache/ignite/streamer/router/StreamerCacheAffinityEventRouter.java
@@ -62,7 +62,7 @@ public class StreamerCacheAffinityEventRouter extends StreamerEventRouterAdapter
 
             assert c != null;
 
-            return c.affinity().mapKeyToNode(e.affinityKey());
+            return ignite.affinity(e.cacheName()).mapKeyToNode(e.affinityKey());
         }
 
         return ignite.cluster().localNode();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/25609651/modules/core/src/main/java/org/gridgain/grid/cache/GridCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/GridCache.java b/modules/core/src/main/java/org/gridgain/grid/cache/GridCache.java
index bc46201..d3a72e6 100644
--- a/modules/core/src/main/java/org/gridgain/grid/cache/GridCache.java
+++ b/modules/core/src/main/java/org/gridgain/grid/cache/GridCache.java
@@ -20,8 +20,6 @@ package org.gridgain.grid.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
-import org.gridgain.grid.cache.affinity.*;
-import org.gridgain.grid.cache.affinity.consistenthash.*;
 import org.gridgain.grid.cache.datastructures.*;
 import org.gridgain.grid.cache.store.*;
 import org.jetbrains.annotations.*;
@@ -41,10 +39,6 @@ import java.util.*;
  *  data based on the optionally passed in arguments.
  * </li>
  * <li>
- *     Method {@link #affinity()} provides {@link GridCacheAffinityFunction} service for information on
- *     data partitioning and mapping keys to grid nodes responsible for caching those keys.
- * </li>
- * <li>
  *     Method {@link #dataStructures()} provides {@link GridCacheDataStructures} service for
  *     creating and working with distributed concurrent data structures, such as
  *     {@link GridCacheAtomicLong}, {@link GridCacheAtomicReference}, {@link GridCacheQueue}, etc.
@@ -95,14 +89,6 @@ public interface GridCache<K, V> extends GridCacheProjection<K, V> {
     public Collection<IgniteTxSynchronization> txSynchronizations();
 
     /**
-     * Gets affinity service to provide information about data partitioning
-     * and distribution.
-     *
-     * @return Cache data affinity service.
-     */
-    public GridCacheAffinity<K> affinity();
-
-    /**
      * Gets data structures service to provide a gateway for creating various
      * distributed data structures similar in APIs to {@code java.util.concurrent} package.
      *
@@ -262,7 +248,7 @@ public interface GridCache<K, V> extends GridCacheProjection<K, V> {
      * the left nodes, and that nodes are restarted before
      * {@link GridCacheConfiguration#getPreloadPartitionedDelay() preloadDelay} expires. To place nodes
      * on the same place in consistent hash ring, use
-     * {@link GridCacheConsistentHashAffinityFunction#setHashIdResolver(GridCacheAffinityNodeHashResolver)} to make sure that
+     * {@link org.apache.ignite.cache.affinity.consistenthash.CacheConsistentHashAffinityFunction#setHashIdResolver(org.apache.ignite.cache.affinity.CacheAffinityNodeHashResolver)} to make sure that
      * a node maps to the same hash ID if re-started.
      * <p>
      * See {@link GridCacheConfiguration#getPreloadPartitionedDelay()} for more information on how to configure

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/25609651/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
index d2ccd49..4ee70bf 100644
--- a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
+++ b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
@@ -22,8 +22,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.portables.PortableObject;
 import org.apache.ignite.spi.indexing.*;
 import org.apache.ignite.transactions.*;
-import org.gridgain.grid.cache.affinity.*;
-import org.gridgain.grid.cache.affinity.consistenthash.*;
+import org.apache.ignite.cache.affinity.*;
 import org.gridgain.grid.cache.cloner.*;
 import org.gridgain.grid.cache.datastructures.*;
 import org.gridgain.grid.cache.eviction.*;
@@ -239,7 +238,7 @@ public class GridCacheConfiguration extends MutableConfiguration {
     private GridCacheStore<?, ?> store;
 
     /** Node group resolver. */
-    private GridCacheAffinityFunction aff;
+    private CacheAffinityFunction aff;
 
     /** Cache mode. */
     private GridCacheMode cacheMode = DFLT_CACHE_MODE;
@@ -308,7 +307,7 @@ public class GridCacheConfiguration extends MutableConfiguration {
     private GridCacheCloner cloner;
 
     /** */
-    private GridCacheAffinityKeyMapper affMapper;
+    private CacheAffinityKeyMapper affMapper;
 
     /** */
     private String indexingSpiName;
@@ -808,7 +807,7 @@ public class GridCacheConfiguration extends MutableConfiguration {
      *
      * @return Key topology resolver to provide mapping from keys to nodes.
      */
-    public GridCacheAffinityFunction getAffinity() {
+    public CacheAffinityFunction getAffinity() {
         return aff;
     }
 
@@ -817,7 +816,7 @@ public class GridCacheConfiguration extends MutableConfiguration {
      *
      * @param aff Cache key affinity.
      */
-    public void setAffinity(GridCacheAffinityFunction aff) {
+    public void setAffinity(CacheAffinityFunction aff) {
         this.aff = aff;
     }
 
@@ -1364,7 +1363,7 @@ public class GridCacheConfiguration extends MutableConfiguration {
      * 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 GridCacheConsistentHashAffinityFunction#setHashIdResolver(GridCacheAffinityNodeHashResolver)}
+     * use {@link org.apache.ignite.cache.affinity.consistenthash.CacheConsistentHashAffinityFunction#setHashIdResolver(org.apache.ignite.cache.affinity.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>
@@ -1429,21 +1428,21 @@ public class GridCacheConfiguration extends MutableConfiguration {
      * on the same node (they will also be backed up on the same nodes as well).
      * <p>
      * If not provided, then default implementation will be used. The default behavior
-     * is described in {@link GridCacheAffinityKeyMapper} documentation.
+     * is described in {@link org.apache.ignite.cache.affinity.CacheAffinityKeyMapper} documentation.
      *
      * @return Mapper to use for affinity key mapping.
      */
-    public GridCacheAffinityKeyMapper getAffinityMapper() {
+    public CacheAffinityKeyMapper getAffinityMapper() {
         return affMapper;
     }
 
     /**
      * Sets custom affinity mapper. If not provided, then default implementation will be used. The default behavior is
-     * described in {@link GridCacheAffinityKeyMapper} documentation.
+     * described in {@link org.apache.ignite.cache.affinity.CacheAffinityKeyMapper} documentation.
      *
      * @param affMapper Affinity mapper.
      */
-    public void setAffinityMapper(GridCacheAffinityKeyMapper affMapper) {
+    public void setAffinityMapper(CacheAffinityKeyMapper affMapper) {
         this.affMapper = affMapper;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/25609651/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheMode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheMode.java b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheMode.java
index 4f471dc..9394b29 100644
--- a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheMode.java
+++ b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheMode.java
@@ -17,7 +17,6 @@
 
 package org.gridgain.grid.cache;
 
-import org.gridgain.grid.cache.affinity.*;
 import org.jetbrains.annotations.*;
 
 /**
@@ -40,7 +39,7 @@ public enum GridCacheMode {
     /**
      * Specifies fully replicated cache behavior. In this mode all the keys are distributed
      * to all participating nodes. User still has affinity control
-     * over subset of nodes for any given key via {@link GridCacheAffinityFunction}
+     * over subset of nodes for any given key via {@link org.apache.ignite.cache.affinity.CacheAffinityFunction}
      * configuration.
      */
     REPLICATED,
@@ -49,7 +48,7 @@ public enum GridCacheMode {
      * Specifies partitioned cache behaviour. In this mode the overall
      * key set will be divided into partitions and all partitions will be split
      * equally between participating nodes. User has affinity
-     * control over key assignment via {@link GridCacheAffinityFunction}
+     * control over key assignment via {@link org.apache.ignite.cache.affinity.CacheAffinityFunction}
      * configuration.
      * <p>
      * Note that partitioned cache is always fronted by local

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/25609651/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheName.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheName.java b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheName.java
index 9044363..42367e2 100644
--- a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheName.java
+++ b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheName.java
@@ -17,8 +17,6 @@
 
 package org.gridgain.grid.cache;
 
-import org.gridgain.grid.cache.affinity.*;
-
 import java.lang.annotation.*;
 import java.util.concurrent.*;
 
@@ -26,14 +24,14 @@ import java.util.concurrent.*;
  * Allows to specify cache name from grid computations. It is used to provide cache name
  * for affinity routing of grid computations, such as {@link org.apache.ignite.compute.ComputeJob}, {@link Runnable},
  * {@link Callable}, or {@link org.apache.ignite.lang.IgniteClosure}. It should be used only in conjunction with
- * {@link GridCacheAffinityKeyMapped @GridCacheAffinityKeyMapped} annotation, and should be attached to a method or field
+ * {@link org.apache.ignite.cache.affinity.CacheAffinityKeyMapped @GridCacheAffinityKeyMapped} annotation, and should be attached to a method or field
  * that provides cache name for the computation. Only one annotation per class
  * is allowed. In the absence of this annotation, the default no-name cache
  * will be used for providing key-to-node affinity.
  * <p>
- * Refer to {@link GridCacheAffinityKeyMapped @GridCacheAffinityKeyMapped} documentation for more information
+ * Refer to {@link org.apache.ignite.cache.affinity.CacheAffinityKeyMapped @GridCacheAffinityKeyMapped} documentation for more information
  * and examples about this annotation.
- * @see GridCacheAffinityKeyMapped
+ * @see org.apache.ignite.cache.affinity.CacheAffinityKeyMapped
  */
 @Documented
 @Retention(RetentionPolicy.RUNTIME)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/25609651/modules/core/src/main/java/org/gridgain/grid/cache/GridCachePreloadMode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/GridCachePreloadMode.java b/modules/core/src/main/java/org/gridgain/grid/cache/GridCachePreloadMode.java
index b87e56d..16d6f4a 100644
--- a/modules/core/src/main/java/org/gridgain/grid/cache/GridCachePreloadMode.java
+++ b/modules/core/src/main/java/org/gridgain/grid/cache/GridCachePreloadMode.java
@@ -17,7 +17,6 @@
 
 package org.gridgain.grid.cache;
 
-import org.gridgain.grid.cache.affinity.*;
 import org.jetbrains.annotations.*;
 
 /**
@@ -27,7 +26,7 @@ import org.jetbrains.annotations.*;
  * explicitly, then {@link GridCacheConfiguration#DFLT_PRELOAD_MODE} is used.
  * <p>
  * Replicated caches will try to load the full set of cache entries from other nodes (or as defined by
- * pluggable {@link GridCacheAffinityFunction}), while partitioned caches will only load the entries for which
+ * pluggable {@link org.apache.ignite.cache.affinity.CacheAffinityFunction}), while partitioned caches will only load the entries for which
  * current node is primary or back up.
  * <p>
  * Note that preload mode only makes sense for {@link GridCacheMode#REPLICATED} and {@link GridCacheMode#PARTITIONED}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/25609651/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheProjection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheProjection.java b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheProjection.java
index 6dbb607..237085c 100644
--- a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheProjection.java
+++ b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheProjection.java
@@ -21,7 +21,6 @@ import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
-import org.gridgain.grid.cache.affinity.*;
 import org.gridgain.grid.cache.query.*;
 import org.gridgain.grid.cache.store.*;
 import org.jetbrains.annotations.*;
@@ -131,13 +130,13 @@ import java.util.concurrent.*;
  * <i>affinity-based</i>, and <i>partitioned-based</i>.
  * <p>
  * With {@code affinity-based-group-locking} the keys are grouped by <i>affinity-key</i>. This means that
- * only keys with identical affinity-key (see {@link GridCacheAffinityKeyMapped}) can participate in the
+ * only keys with identical affinity-key (see {@link org.apache.ignite.cache.affinity.CacheAffinityKeyMapped}) can participate in the
  * transaction, and only one lock on the <i>affinity-key</i> will be acquired for the whole transaction.
  * {@code Affinity-group-locked} transactions are started via
  * {@link #txStartAffinity(Object, IgniteTxConcurrency, IgniteTxIsolation, long, int)} method.
  * <p>
  * With {@code partition-based-group-locking} the keys are grouped by partition ID. This means that
- * only keys belonging to identical partition (see {@link GridCacheAffinity#partition(Object)}) can participate in the
+ * only keys belonging to identical partition (see {@link org.apache.ignite.IgniteCacheAffinity#partition(Object)}) can participate in the
  * transaction, and only one lock on the whole partition will be acquired for the whole transaction.
  * {@code Partition-group-locked} transactions are started via
  * {@link #txStartPartition(int, IgniteTxConcurrency, IgniteTxIsolation, long, int)} method.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/25609651/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinity.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinity.java b/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinity.java
deleted file mode 100644
index 5905890..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinity.java
+++ /dev/null
@@ -1,262 +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.grid.cache.affinity;
-
-import org.apache.ignite.cluster.*;
-import org.gridgain.grid.cache.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-/**
- * Provides affinity information to detect which node is primary and which nodes are
- * backups for a partitioned cache. You can get an instance of this interface by calling
- * {@code GridCache.affinity()} method.
- * <p>
- * Mapping of a key to a node is a three-step operation. First step will get an affinity key for given key
- * using {@link GridCacheAffinityKeyMapper}. If mapper is not specified, the original key will be used. Second step
- * will map affinity key to partition using {@link GridCacheAffinityFunction#partition(Object)} method. Third step
- * will map obtained partition to nodes for current grid topology version.
- * <p>
- * Interface provides various {@code 'mapKeysToNodes(..)'} methods which provide node affinity mapping for
- * given keys. All {@code 'mapKeysToNodes(..)'} methods are not transactional and will not enlist
- * keys into ongoing transaction.
- */
-public interface GridCacheAffinity<K> {
-    /**
-     * Gets number of partitions in cache according to configured affinity function.
-     *
-     * @return Number of cache partitions.
-     * @see GridCacheAffinityFunction
-     * @see GridCacheConfiguration#getAffinity()
-     * @see GridCacheConfiguration#setAffinity(GridCacheAffinityFunction)
-     */
-    public int partitions();
-
-    /**
-     * Gets partition id for the given key.
-     *
-     * @param key Key to get partition id for.
-     * @return Partition id.
-     * @see GridCacheAffinityFunction
-     * @see GridCacheConfiguration#getAffinity()
-     * @see GridCacheConfiguration#setAffinity(GridCacheAffinityFunction)
-     */
-    public int partition(K key);
-
-    /**
-     * Returns {@code true} if given node is the primary node for given key.
-     * To check if local node is primary for given key, pass
-     * {@link org.apache.ignite.Ignite#localNode()} as first parameter.
-     *
-     * @param n Node to check.
-     * @param key Key to check.
-     * @return {@code True} if local node is the primary node for given key.
-     */
-    public boolean isPrimary(ClusterNode n, K key);
-
-    /**
-     * Returns {@code true} if local node is one of the backup nodes for given key.
-     * To check if local node is primary for given key, pass {@link org.apache.ignite.Ignite#localNode()}
-     * as first parameter.
-     *
-     * @param n Node to check.
-     * @param key Key to check.
-     * @return {@code True} if local node is one of the backup nodes for given key.
-     */
-    public boolean isBackup(ClusterNode n, K key);
-
-    /**
-     * Returns {@code true} if local node is primary or one of the backup nodes
-     * for given key. To check if local node is primary or backup for given key, pass
-     * {@link org.apache.ignite.Ignite#localNode()} as first parameter.
-     * <p>
-     * This method is essentially equivalent to calling
-     * <i>"{@link #isPrimary(org.apache.ignite.cluster.ClusterNode, Object)} || {@link #isBackup(org.apache.ignite.cluster.ClusterNode, Object)})"</i>,
-     * however it is more efficient as it makes both checks at once.
-     *
-     * @param n Node to check.
-     * @param key Key to check.
-     * @return {@code True} if local node is primary or backup for given key.
-     */
-    public boolean isPrimaryOrBackup(ClusterNode n, K key);
-
-    /**
-     * Gets partition ids for which nodes of the given projection has primary
-     * ownership.
-     * <p>
-     * Note that since {@link org.apache.ignite.cluster.ClusterNode} implements {@link org.apache.ignite.cluster.ClusterGroup},
-     * to find out primary partitions for a single node just pass
-     * a single node into this method.
-     * <p>
-     * This method may return an empty array if none of nodes in the projection
-     * have nearOnly disabled.
-     *
-     * @param n Grid node.
-     * @return Partition ids for which given projection has primary ownership.
-     * @see GridCacheAffinityFunction
-     * @see GridCacheConfiguration#getAffinity()
-     * @see GridCacheConfiguration#setAffinity(GridCacheAffinityFunction)
-     */
-    public int[] primaryPartitions(ClusterNode n);
-
-    /**
-     * Gets partition ids for which nodes of the given projection has backup
-     * ownership. Note that you can find a back up at a certain level, e.g.
-     * {@code first} backup or {@code third} backup by specifying the
-     * {@code 'levels} parameter. If no {@code 'level'} is specified then
-     * all backup partitions are returned.
-     * <p>
-     * Note that since {@link org.apache.ignite.cluster.ClusterNode} implements {@link org.apache.ignite.cluster.ClusterGroup},
-     * to find out backup partitions for a single node, just pass that single
-     * node into this method.
-     * <p>
-     * This method may return an empty array if none of nodes in the projection
-     * have nearOnly disabled.
-     *
-     * @param n Grid node.
-     * @return Partition ids for which given projection has backup ownership.
-     * @see GridCacheAffinityFunction
-     * @see GridCacheConfiguration#getAffinity()
-     * @see GridCacheConfiguration#setAffinity(GridCacheAffinityFunction)
-     */
-    public int[] backupPartitions(ClusterNode n);
-
-    /**
-     * Gets partition ids for which nodes of the given projection has ownership
-     * (either primary or backup).
-     * <p>
-     * Note that since {@link org.apache.ignite.cluster.ClusterNode} implements {@link org.apache.ignite.cluster.ClusterGroup},
-     * to find out all partitions for a single node, just pass that single
-     * node into this method.
-     * <p>
-     * This method may return an empty array if none of nodes in the projection
-     * have nearOnly disabled.
-     *
-     * @param n Grid node.
-     * @return Partition ids for which given projection has ownership.
-     * @see GridCacheAffinityFunction
-     * @see GridCacheConfiguration#getAffinity()
-     * @see GridCacheConfiguration#setAffinity(GridCacheAffinityFunction)
-     */
-    public int[] allPartitions(ClusterNode n);
-
-    /**
-     * Maps passed in key to a key which will be used for node affinity. The affinity
-     * key may be different from actual key if some field in the actual key was
-     * designated for affinity mapping via {@link GridCacheAffinityKeyMapped} annotation
-     * or if a custom {@link GridCacheAffinityKeyMapper} was configured.
-     *
-     * @param key Key to map.
-     * @return Key to be used for node-to-affinity mapping (may be the same
-     *      key as passed in).
-     */
-    public Object affinityKey(K key);
-
-    /**
-     * This method provides ability to detect which keys are mapped to which nodes.
-     * Use it to determine which nodes are storing which keys prior to sending
-     * jobs that access these keys.
-     * <p>
-     * This method works as following:
-     * <ul>
-     * <li>For local caches it returns only local node mapped to all keys.</li>
-     * <li>
-     *      For fully replicated caches {@link GridCacheAffinityFunction} is
-     *      used to determine which keys are mapped to which nodes.
-     * </li>
-     * <li>For partitioned caches, the returned map represents node-to-key affinity.</li>
-     * </ul>
-     *
-     * @param keys Keys to map to nodes.
-     * @return Map of nodes to keys or empty map if there are no alive nodes for this cache.
-     */
-    public Map<ClusterNode, Collection<K>> mapKeysToNodes(@Nullable Collection<? extends K> keys);
-
-    /**
-     * This method provides ability to detect to which primary node the given key
-     * is mapped. Use it to determine which nodes are storing which keys prior to sending
-     * jobs that access these keys.
-     * <p>
-     * This method works as following:
-     * <ul>
-     * <li>For local caches it returns only local node ID.</li>
-     * <li>
-     *      For fully replicated caches first node ID returned by {@link GridCacheAffinityFunction}
-     *      is returned.
-     * </li>
-     * <li>For partitioned caches, primary node for the given key is returned.</li>
-     * </ul>
-     *
-     * @param key Keys to map to a node.
-     * @return Primary node for the key or {@code null} if there are no alive nodes for this cache.
-     */
-    @Nullable public ClusterNode mapKeyToNode(K key);
-
-    /**
-     * Gets primary and backup nodes for the key. Note that primary node is always
-     * first in the returned collection.
-     * <p>
-     * If there are only cache nodes in the projection with
-     * {@link GridCacheConfiguration#getDistributionMode()} property set to {@code NEAR_ONLY}, then this
-     * method will return an empty collection.
-     *
-     * @param key Key to get affinity nodes for.
-     * @return Collection of primary and backup nodes for the key with primary node
-     *      always first, or an empty collection if this projection contains only nodes with
-     *      {@link GridCacheConfiguration#getDistributionMode()} property set to {@code NEAR_ONLY}.
-     */
-    public Collection<ClusterNode> mapKeyToPrimaryAndBackups(K key);
-
-    /**
-     * Gets primary node for the given partition.
-     *
-     * @param part Partition id.
-     * @return Primary node for the given partition.
-     * @see GridCacheAffinityFunction
-     * @see GridCacheConfiguration#getAffinity()
-     * @see GridCacheConfiguration#setAffinity(GridCacheAffinityFunction)
-     */
-    public ClusterNode mapPartitionToNode(int part);
-
-    /**
-     * Gets primary nodes for the given partitions.
-     *
-     * @param parts Partition ids.
-     * @return Mapping of given partitions to their primary nodes.
-     * @see GridCacheAffinityFunction
-     * @see GridCacheConfiguration#getAffinity()
-     * @see GridCacheConfiguration#setAffinity(GridCacheAffinityFunction)
-     */
-    public Map<Integer, ClusterNode> mapPartitionsToNodes(Collection<Integer> parts);
-
-    /**
-     * Gets primary and backup nodes for partition. Note that primary node is always
-     * first in the returned collection.
-     * <p>
-     * If there are only cache nodes in the projection with
-     * {@link GridCacheConfiguration#getDistributionMode()} property set to {@code NEAR_ONLY}, then this
-     * method will return an empty collection.
-     *
-     * @param part Partition to get affinity nodes for.
-     * @return Collection of primary and backup nodes for partition with primary node
-     *      always first, or an empty collection if this projection contains only nodes with
-     *      {@link GridCacheConfiguration#getDistributionMode()} property set to {@code NEAR_ONLY}.
-     */
-    public Collection<ClusterNode> mapPartitionToPrimaryAndBackups(int part);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/25609651/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityFunction.java b/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityFunction.java
deleted file mode 100644
index f1d44d9..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityFunction.java
+++ /dev/null
@@ -1,111 +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.grid.cache.affinity;
-
-import org.apache.ignite.cluster.*;
-import org.gridgain.grid.cache.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Cache key affinity which maps keys to nodes. This interface is utilized for
- * both, replicated and partitioned caches. Cache affinity can be configured
- * for individual caches via {@link GridCacheConfiguration#getAffinity()} method.
- * <p>
- * Whenever a key is given to cache, it is first passed to a pluggable
- * {@link GridCacheAffinityKeyMapper} which may potentially map this key to an alternate
- * key which should be used for affinity. The key returned from
- * {@link GridCacheAffinityKeyMapper#affinityKey(Object)} method is then passed to
- * {@link #partition(Object) partition(Object)} method to find out the partition for the key.
- * On each topology change, partition-to-node mapping is calculated using
- * {@link #assignPartitions(GridCacheAffinityFunctionContext)} method, which assigns a collection
- * of nodes to each partition.
- * This collection of nodes is used for node affinity. In {@link GridCacheMode#REPLICATED REPLICATED}
- * cache mode the key will be cached on all returned nodes; generally, all caching nodes
- * participate in caching every key in replicated mode. In {@link GridCacheMode#PARTITIONED PARTITIONED}
- * mode, only primary and backup nodes are returned with primary node always in the
- * first position. So if there is {@code 1} backup node, then the returned collection will
- * have {@code 2} nodes in it - {@code primary} node in first position, and {@code backup}
- * node in second.
- * <p>
- * For more information about cache affinity and examples refer to {@link GridCacheAffinityKeyMapper} and
- * {@link GridCacheAffinityKeyMapped @GridCacheAffinityKeyMapped} documentation.
- * @see GridCacheAffinityKeyMapped
- * @see GridCacheAffinityKeyMapper
- */
-public interface GridCacheAffinityFunction extends Serializable {
-    /**
-     * Resets cache affinity to its initial state. This method will be called by
-     * the system any time the affinity has been sent to remote node where
-     * it has to be reinitialized. If your implementation of affinity function
-     * has no initialization logic, leave this method empty.
-     */
-    public void reset();
-
-    /**
-     * Gets total number of partitions available. All caches should always provide
-     * correct partition count which should be the same on all participating nodes.
-     * Note that partitions should always be numbered from {@code 0} inclusively to
-     * {@code N} exclusively without any gaps.
-     *
-     * @return Total partition count.
-     */
-    public int partitions();
-
-    /**
-     * Gets partition number for a given key starting from {@code 0}. Partitioned caches
-     * should make sure that keys are about evenly distributed across all partitions
-     * from {@code 0} to {@link #partitions() partition count} for best performance.
-     * <p>
-     * Note that for fully replicated caches it is possible to segment key sets among different
-     * grid node groups. In that case each node group should return a unique partition
-     * number. However, unlike partitioned cache, mappings of keys to nodes in
-     * replicated caches are constant and a node cannot migrate from one partition
-     * to another.
-     *
-     * @param key Key to get partition for.
-     * @return Partition number for a given key.
-     */
-    public int partition(Object key);
-
-    /**
-     * Gets affinity nodes for a partition. In case of replicated cache, all returned
-     * nodes are updated in the same manner. In case of partitioned cache, the returned
-     * list should contain only the primary and back up nodes with primary node being
-     * always first.
-     * <p>
-     * Note that partitioned affinity must obey the following contract: given that node
-     * <code>N</code> is primary for some key <code>K</code>, if any other node(s) leave
-     * grid and no node joins grid, node <code>N</code> will remain primary for key <code>K</code>.
-     *
-     * @param affCtx Affinity function context. Will provide all required information to calculate
-     *      new partition assignments.
-     * @return Unmodifiable list indexed by partition number. Each element of array is a collection in which
-     *      first node is a primary node and other nodes are backup nodes.
-     */
-    public List<List<ClusterNode>> assignPartitions(GridCacheAffinityFunctionContext affCtx);
-
-    /**
-     * Removes node from affinity. This method is called when it is safe to remove left node from
-     * affinity mapping.
-     *
-     * @param nodeId ID of node to remove.
-     */
-    public void removeNode(UUID nodeId);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/25609651/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityFunctionContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityFunctionContext.java b/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityFunctionContext.java
deleted file mode 100644
index 3d1a14c..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityFunctionContext.java
+++ /dev/null
@@ -1,71 +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.grid.cache.affinity;
-
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.events.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-/**
- * Affinity function context. This context is passed to {@link GridCacheAffinityFunction} for
- * partition reassignment on every topology change event.
- */
-public interface GridCacheAffinityFunctionContext {
-    /**
-     * Gets affinity assignment for given partition on previous topology version. First node in returned list is
-     * a primary node, other nodes are backups.
-     *
-     * @param part Partition to get previous assignment for.
-     * @return List of nodes assigned to given partition on previous topology version or {@code null}
-     *      if this information is not available.
-     */
-    @Nullable public List<ClusterNode> previousAssignment(int part);
-
-    /**
-     * Gets number of backups for new assignment.
-     *
-     * @return Number of backups for new assignment.
-     */
-    public int backups();
-
-    /**
-     * Gets current topology snapshot. Snapshot will contain only nodes on which particular cache is configured.
-     * List of passed nodes is guaranteed to be sorted in a same order on all nodes on which partition assignment
-     * is performed.
-     *
-     * @return Cache topology snapshot.
-     */
-    public List<ClusterNode> currentTopologySnapshot();
-
-    /**
-     * Gets current topology version number.
-     *
-     * @return Current topology version number.
-     */
-    public long currentTopologyVersion();
-
-    /**
-     * Gets discovery event caused topology change.
-     *
-     * @return Discovery event caused latest topology change or {@code null} if this information is
-     *      not available.
-     */
-    @Nullable public IgniteDiscoveryEvent discoveryEvent();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/25609651/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKey.java b/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKey.java
deleted file mode 100644
index cab9264..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKey.java
+++ /dev/null
@@ -1,189 +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.grid.cache.affinity;
-
-import org.gridgain.grid.kernal.processors.cache.*;
-import org.gridgain.grid.util.typedef.internal.*;
-import org.gridgain.grid.util.tostring.*;
-
-import java.io.*;
-
-/**
- * Optional wrapper for cache keys to provide support
- * for custom affinity mapping. The value returned by
- * {@link #affinityKey(Object)} method will be used for key-to-node
- * affinity.
- * <p>
- * Note that the {@link #equals(Object)} and {@link #hashCode()} methods
- * delegate directly to the wrapped cache key provided by {@link #key()}
- * method.
- * <p>
- * This class is optional and does not have to be used. It only provides
- * extra convenience whenever custom affinity mapping is required. Here is
- * an example of how {@code Person} objects can be collocated with
- * {@code Company} objects they belong to:
- * <pre name="code" class="java">
- * Object personKey = new GridCacheAffinityKey(myPersonId, myCompanyId);
- *
- * // Both, the company and the person objects will be cached on the same node.
- * cache.put(myCompanyId, new Company(..));
- * cache.put(personKey, new Person(..));
- * </pre>
- * <p>
- * For more information and examples of cache affinity refer to
- * {@link GridCacheAffinityKeyMapper} and {@link GridCacheAffinityKeyMapped @GridCacheAffinityKeyMapped}
- * documentation.
- * @see GridCacheAffinityKeyMapped
- * @see GridCacheAffinityKeyMapper
- * @see GridCacheAffinityFunction
- */
-public class GridCacheAffinityKey<K> implements Externalizable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Key. */
-    @GridToStringInclude
-    private K key;
-
-    /** Affinity key. */
-    @GridToStringInclude
-    private Object affKey;
-
-    /**
-     * Empty constructor.
-     */
-    public GridCacheAffinityKey() {
-        // No-op.
-    }
-
-    /**
-     * Initializes key wrapper for a given key. If affinity key
-     * is not initialized, then this key will be used for affinity.
-     *
-     * @param key Key.
-     */
-    public GridCacheAffinityKey(K key) {
-        A.notNull(key, "key");
-
-        this.key = key;
-    }
-
-    /**
-     * Initializes key together with its affinity key counter-part.
-     *
-     * @param key Key.
-     * @param affKey Affinity key.
-     */
-    public GridCacheAffinityKey(K key, Object affKey) {
-        A.notNull(key, "key");
-
-        this.key = key;
-        this.affKey = affKey;
-    }
-
-    /**
-     * Gets wrapped key.
-     *
-     * @return Wrapped key.
-     */
-    public K key() {
-        return key;
-    }
-
-    /**
-     * Sets wrapped key.
-     *
-     * @param key Wrapped key.
-     */
-    public void key(K key) {
-        this.key = key;
-    }
-
-    /**
-     * Gets affinity key to use for affinity mapping. If affinity key is not provided,
-     * then {@code key} value will be returned.
-     * <p>
-     * This method is annotated with {@link GridCacheAffinityKeyMapped} and will be picked up
-     * by {@link GridCacheDefaultAffinityKeyMapper} automatically.
-     *
-     * @return Affinity key to use for affinity mapping.
-     */
-    @GridCacheAffinityKeyMapped
-    @SuppressWarnings({"unchecked"})
-    public <T> T affinityKey() {
-        A.notNull(key, "key");
-
-        return (T)(affKey == null ? key : affKey);
-    }
-
-    /**
-     * Sets affinity key to use for affinity mapping. If affinity key is not provided,
-     * then {@code key} value will be returned.
-     *
-     * @param affKey Affinity key to use for affinity mapping.
-     */
-    public void affinityKey(Object affKey) {
-        this.affKey = affKey;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeObject(key);
-        out.writeObject(affKey);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        key = (K)in.readObject();
-        affKey = in.readObject();
-    }
-
-    /**
-     * Hash code implementation which delegates to the underlying {@link #key()}. Note, however,
-     * that different subclasses of {@code GridCacheAffinityKey} will produce different hash codes.
-     * <p>
-     * Users should override this method if different behavior is desired.
-     *
-     * @return Hash code.
-     */
-    @Override public int hashCode() {
-        A.notNull(key, "key");
-
-        return 31 * key.hashCode() + getClass().getName().hashCode();
-    }
-
-    /**
-     * Equality check which delegates to the underlying key equality. Note, however, that
-     * different subclasses of {@code GridCacheAffinityKey} will never be equal.
-     * <p>
-     * Users should override this method if different behavior is desired.
-     *
-     * @param obj Object to check for equality.
-     * @return {@code True} if objects are equal.
-     */
-    @Override public boolean equals(Object obj) {
-        A.notNull(key, "key");
-
-        return obj != null && getClass() == obj.getClass() && key.equals(((GridCacheAffinityKey)obj).key);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridCacheAffinityKey.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/25609651/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKeyMapped.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKeyMapped.java b/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKeyMapped.java
deleted file mode 100644
index 311ef85..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKeyMapped.java
+++ /dev/null
@@ -1,158 +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.grid.cache.affinity;
-
-import org.gridgain.grid.cache.*;
-
-import java.lang.annotation.*;
-import java.util.concurrent.*;
-
-/**
- * Optional annotation to specify custom key-to-node affinity. Affinity key is a key
- * which will be used to determine a node on which given cache key will be stored. This
- * annotation allows to mark a field or a method in the cache key object that will be
- * used as an affinity key (instead of the entire cache key object that is used for
- * affinity by default). Note that a class can have only one field or method annotated
- * with {@code @GridCacheAffinityKeyMapped} annotation.
- * <p>
- * One of the major use cases for this annotation is the routing of grid computations
- * to the nodes where the data for this computation is cached, the concept
- * otherwise known as {@code Collocation Of Computations And Data}.
- * <p>
- * <h1 class="header">Mapping Cache Keys</h1>
- * The default implementation of {@link GridCacheAffinityKeyMapper}, which will be used
- * if no explicit affinity mapper is specified in cache configuration, will first look
- * for any field or method annotated with {@code @GridCacheAffinityKeyMapped} annotation.
- * If such field or method is not found, then the cache key itself will be used for
- * key-to-node affinity (this means that all objects with the same cache key will always
- * be routed to the same node). If such field or method is found, then the value of this
- * field or method will be used for key-to-node affinity. This allows to specify alternate
- * affinity key, other than the cache key itself, whenever needed.
- * <p>
- * For example, if a {@code Person} object is always accessed together with a {@code Company} object
- * for which this person is an employee, then for better performance and scalability it makes sense to
- * collocate {@code Person} objects together with their {@code Company} object when storing them in
- * cache. To achieve that, cache key used to cache {@code Person} objects should have a field or method
- * annotated with {@code @GridCacheAffinityKeyMapped} annotation, which will provide the value of
- * the company key for which that person works, like so:
- * <pre name="code" class="java">
- * public class PersonKey {
- *     // Person ID used to identify a person.
- *     private String personId;
- *
- *     // Company ID which will be used for affinity.
- *     &#64;GridCacheAffinityKeyMapped
- *     private String companyId;
- *     ...
- * }
- * ...
- * // Instantiate person keys.
- * Object personKey1 = new PersonKey("myPersonId1", "myCompanyId");
- * Object personKey2 = new PersonKey("myPersonId2", "myCompanyId");
- *
- * // Both, the company and the person objects will be cached on the same node.
- * cache.put("myCompanyId", new Company(..));
- * cache.put(personKey1, new Person(..));
- * cache.put(personKey2, new Person(..));
- * </pre>
- * <p>
- * <h2 class="header">GridCacheAffinityKey</h2>
- * For convenience, you can also optionally use {@link GridCacheAffinityKey} class. Here is how a
- * {@code PersonKey} defined above would look using {@link GridCacheAffinityKey}:
- * <pre name="code" class="java">
- * Object personKey1 = new GridCacheAffinityKey("myPersonId1", "myCompanyId");
- * Object personKey2 = new GridCacheAffinityKey("myPersonId2", "myCompanyId");
- *
- * // Both, the company and the person objects will be cached on the same node.
- * cache.put(myCompanyId, new Company(..));
- * cache.put(personKey1, new Person(..));
- * cache.put(personKey2, new Person(..));
- * </pre>
- * <p>
- * <h1 class="header">Collocating Computations And Data</h1>
- * It is also possible to route computations to the nodes where the data is cached. This concept
- * is otherwise known as {@code Collocation Of Computations And Data}. In this case,
- * {@code @GridCacheAffinityKeyMapped} annotation allows to specify a routing affinity key for a
- * {@link org.apache.ignite.compute.ComputeJob} or any other grid computation, such as {@link Runnable}, {@link Callable}, or
- * {@link org.apache.ignite.lang.IgniteClosure}. It should be attached to a method or field that provides affinity key
- * for the computation. Only one annotation per class is allowed. Whenever such annotation is detected,
- * then {@link org.apache.ignite.spi.loadbalancing.LoadBalancingSpi} will be bypassed, and computation will be routed to the grid node
- * where the specified affinity key is cached. You can also use optional {@link GridCacheName @GridCacheName}
- * annotation whenever non-default cache name needs to be specified.
- * <p>
- * Here is how this annotation can be used to route a job to a node where Person object
- * is cached with ID "1234":
- * <pre name="code" class="java">
- * G.grid().run(new Runnable() {
- *     // This annotation is optional. If omitted, then default
- *     // no-name cache will be used.
- *     &#64;GridCacheName
- *     private String cacheName = "myCache";
- *
- *     // This annotation specifies that computation should be routed
- *     // precisely to the node where key '1234' is cached.
- *     &#64;GridCacheAffinityKeyMapped
- *     private String personKey = "1234";
- *
- *     &#64;Override public void run() {
- *         // Some computation logic here.
- *         ...
- *     }
- * };
- * </pre>
- * The same can be achieved by annotating method instead of field as follows:
- * <pre name="code" class="java">
- * G.grid().run(new Runnable() {
- *     &#64;Override public void run() {
- *         // Some computation logic here.
- *         ...
- *     }
- *
- *     // This annotation is optional. If omitted, then default
- *     // no-name cache will be used.
- *     &#64;GridCacheName
- *     public String cacheName() {
- *         return "myCache";
- *     }
- *
- *     // This annotation specifies that computation should be routed
- *     // precisely to the node where key '1234' is cached.
- *     &#64;GridCacheAffinityKeyMapped
- *     public String personKey() {
- *         return "1234";
- *     }
- * };
- * </pre>
- * <p>
- * For more information about cache affinity also see {@link GridCacheAffinityKeyMapper} and
- * {@link GridCacheAffinityFunction} documentation.
- * Affinity for a key can be found from any node, regardless of whether it has cache started
- * or not. If cache is not started, affinity function will be fetched from the remote node
- * which does have the cache running.
- *
- * @see GridCacheName
- * @see GridCacheAffinityFunction
- * @see GridCacheAffinityKeyMapper
- * @see GridCacheAffinityKey
- */
-@Documented
-@Retention(RetentionPolicy.RUNTIME)
-@Target({ElementType.FIELD, ElementType.METHOD})
-public @interface GridCacheAffinityKeyMapped {
-    // No-op.
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/25609651/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKeyMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKeyMapper.java b/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKeyMapper.java
deleted file mode 100644
index b8b7037..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKeyMapper.java
+++ /dev/null
@@ -1,64 +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.grid.cache.affinity;
-
-import org.gridgain.grid.cache.*;
-
-import java.io.*;
-
-/**
- * Affinity mapper which maps cache key to an affinity key. Affinity key is a key which will be
- * used to determine a node on which this key will be cached. Every cache key will first be passed
- * through {@link #affinityKey(Object)} method, and the returned value of this method
- * will be given to {@link GridCacheAffinityFunction} implementation to find out key-to-node affinity.
- * <p>
- * The default implementation, which will be used if no explicit affinity mapper is specified
- * in cache configuration, will first look for any field or method annotated with
- * {@link GridCacheAffinityKeyMapped @GridCacheAffinityKeyMapped} annotation. If such field or method
- * is not found, then the cache key itself will be returned from {@link #affinityKey(Object) affinityKey(Object)}
- * method (this means that all objects with the same cache key will always be routed to the same node).
- * If such field or method is found, then the value of this field or method will be returned from
- * {@link #affinityKey(Object) affinityKey(Object)} method. This allows to specify alternate affinity key, other
- * than the cache key itself, whenever needed.
- * <p>
- * A custom (other than default) affinity mapper can be provided
- * via {@link GridCacheConfiguration#getAffinityMapper()} configuration property.
- * <p>
- * For more information on affinity mapping and examples refer to {@link GridCacheAffinityFunction} and
- * {@link GridCacheAffinityKeyMapped @GridCacheAffinityKeyMapped} documentation.
- * @see GridCacheAffinityFunction
- * @see GridCacheAffinityKeyMapped
- */
-public interface GridCacheAffinityKeyMapper extends Serializable {
-    /**
-     * Maps passed in key to an alternate key which will be used for node affinity.
-     *
-     * @param key Key to map.
-     * @return Key to be used for node-to-affinity mapping (may be the same
-     *      key as passed in).
-     */
-    public Object affinityKey(Object key);
-
-    /**
-     * Resets cache affinity mapper to its initial state. This method will be called by
-     * the system any time the affinity mapper has been sent to remote node where
-     * it has to be reinitialized. If your implementation of affinity mapper
-     * has no initialization logic, leave this method empty.
-     */
-    public void reset();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/25609651/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityNodeAddressHashResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityNodeAddressHashResolver.java b/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityNodeAddressHashResolver.java
deleted file mode 100644
index e646267..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityNodeAddressHashResolver.java
+++ /dev/null
@@ -1,39 +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.grid.cache.affinity;
-
-import org.apache.ignite.cluster.*;
-import org.gridgain.grid.util.typedef.internal.*;
-
-/**
- * Node hash resolver which uses {@link org.apache.ignite.cluster.ClusterNode#consistentId()} as alternate hash value.
- */
-public class GridCacheAffinityNodeAddressHashResolver implements GridCacheAffinityNodeHashResolver {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** {@inheritDoc} */
-    @Override public Object resolve(ClusterNode node) {
-        return node.consistentId();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridCacheAffinityNodeAddressHashResolver.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/25609651/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityNodeHashResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityNodeHashResolver.java b/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityNodeHashResolver.java
deleted file mode 100644
index dc79f66..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityNodeHashResolver.java
+++ /dev/null
@@ -1,43 +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.grid.cache.affinity;
-
-import org.apache.ignite.cluster.*;
-
-import java.io.*;
-
-/**
- * Resolver which is used to provide node hash value for affinity function.
- * <p>
- * Node IDs constantly change when nodes get restarted, which causes affinity mapping to change between restarts,
- * and hence causing redundant repartitioning. Providing an alternate node hash value, which survives node restarts,
- * will help to map keys to the same nodes whenever possible.
- * <p>
- * Note that on case clients exist they will query this object from the server and use it for affinity calculation.
- * Therefore you must ensure that server and clients can marshal and unmarshal this object in portable format,
- * i.e. all parties have object class(es) configured as portable.
- */
-public interface GridCacheAffinityNodeHashResolver extends Serializable {
-    /**
-     * Resolve alternate hash value for the given Grid node.
-     *
-     * @param node Grid node.
-     * @return Resolved hash ID.
-     */
-    public Object resolve(ClusterNode node);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/25609651/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityNodeIdHashResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityNodeIdHashResolver.java b/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityNodeIdHashResolver.java
deleted file mode 100644
index d401afa..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityNodeIdHashResolver.java
+++ /dev/null
@@ -1,40 +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.grid.cache.affinity;
-
-import org.apache.ignite.cluster.*;
-import org.gridgain.grid.util.typedef.internal.*;
-
-/**
- * Node hash resolver which uses generated node ID as node hash value. As new node ID is generated
- * on each node start, this resolver do not provide ability to map keys to the same nodes after restart.
- */
-public class GridCacheAffinityNodeIdHashResolver implements GridCacheAffinityNodeHashResolver {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** {@inheritDoc} */
-    @Override public Object resolve(ClusterNode node) {
-        return node.id();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridCacheAffinityNodeIdHashResolver.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/25609651/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheCentralizedAffinityFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheCentralizedAffinityFunction.java b/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheCentralizedAffinityFunction.java
deleted file mode 100644
index b20680c..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheCentralizedAffinityFunction.java
+++ /dev/null
@@ -1,31 +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.grid.cache.affinity;
-
-import java.lang.annotation.*;
-
-/**
- * Annotation marker which identifies affinity function that must be calculated on one centralized node
- * instead of independently on each node. In many cases it happens because it requires previous affinity state
- * in order to calculate new one.
- */
-@Retention(RetentionPolicy.RUNTIME)
-@Target(ElementType.TYPE)
-public @interface GridCacheCentralizedAffinityFunction {
-
-}