http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinity.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinity.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinity.java
deleted file mode 100644
index 9f65dc7..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinity.java
+++ /dev/null
@@ -1,805 +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.fair;
-
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.events.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Fair affinity function which tries to ensure that all nodes get equal 
number of partitions with
- * minimum amount of reassignments between existing nodes.
- * <p>
- * Cache affinity can be configured for individual caches via {@link 
CacheConfiguration#getAffinity()} method.
- */
-@GridCacheCentralizedAffinityFunction
-public class GridCachePartitionFairAffinity implements 
GridCacheAffinityFunction {
-    /** Default partition count. */
-    public static final int DFLT_PART_CNT = 256;
-
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Ascending comparator. */
-    private static final Comparator<PartitionSet> ASC_CMP = new 
PartitionSetComparator(false);
-
-    /** Descending comparator. */
-    private static final Comparator<PartitionSet> DESC_CMP = new 
PartitionSetComparator(true);
-
-    /** */
-    private int parts;
-
-    /**
-     * Creates fair affinity with default partition count.
-     */
-    public GridCachePartitionFairAffinity() {
-        this(DFLT_PART_CNT);
-    }
-
-    /**
-     * @param parts Number of partitions.
-     */
-    public GridCachePartitionFairAffinity(int parts) {
-        this.parts = parts;
-    }
-
-    /** {@inheritDoc} */
-    @Override public List<List<ClusterNode>> 
assignPartitions(GridCacheAffinityFunctionContext ctx) {
-        List<ClusterNode> topSnapshot = ctx.currentTopologySnapshot();
-
-        if (topSnapshot.size() == 1) {
-            ClusterNode primary = topSnapshot.get(0);
-
-            List<List<ClusterNode>> assignments = new ArrayList<>(parts);
-
-            for (int i = 0; i < parts; i++)
-                assignments.add(Collections.singletonList(primary));
-
-            return assignments;
-        }
-
-        IgniteBiTuple<List<List<ClusterNode>>, Map<UUID, PartitionSet>> cp = 
createCopy(ctx, topSnapshot);
-
-        List<List<ClusterNode>> assignment = cp.get1();
-
-        int tiers = Math.min(ctx.backups() + 1, topSnapshot.size());
-
-        // Per tier pending partitions.
-        Map<Integer, Queue<Integer>> pendingParts = new HashMap<>();
-
-        FullAssignmentMap fullMap = new FullAssignmentMap(tiers, assignment, 
topSnapshot);
-
-        for (int tier = 0; tier < tiers; tier++) {
-            // Check if this is a new tier and add pending partitions.
-            Queue<Integer> pending = pendingParts.get(tier);
-
-            for (int part = 0; part < parts; part++) {
-                if (fullMap.assignments.get(part).size() < tier + 1) {
-                    if (pending == null) {
-                        pending = new LinkedList<>();
-
-                        pendingParts.put(tier, pending);
-                    }
-
-                    if (!pending.contains(part))
-                        pending.add(part);
-
-                }
-            }
-
-            // Assign pending partitions, if any.
-            assignPending(tier, pendingParts, fullMap, topSnapshot);
-
-            // Balance assignments.
-            balance(tier, pendingParts, fullMap, topSnapshot);
-        }
-
-        return fullMap.assignments;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void reset() {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public int partitions() {
-        return parts;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int partition(Object key) {
-        return U.safeAbs(hash(key.hashCode())) % parts;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeNode(UUID nodeId) {
-        // No-op.
-    }
-
-    /**
-     * Assigns pending (unassigned) partitions to nodes.
-     *
-     * @param tier Tier to assign (0 is primary, 1 - 1st backup,...).
-     * @param pendingMap Pending partitions per tier.
-     * @param fullMap Full assignment map to modify.
-     * @param topSnapshot Topology snapshot.
-     */
-    private void assignPending(int tier, Map<Integer, Queue<Integer>> 
pendingMap, FullAssignmentMap fullMap,
-        List<ClusterNode> topSnapshot) {
-        Queue<Integer> pending = pendingMap.get(tier);
-
-        if (F.isEmpty(pending))
-            return;
-
-        int idealPartCnt = parts / topSnapshot.size();
-
-        Map<UUID, PartitionSet> tierMapping = fullMap.tierMapping(tier);
-
-        PrioritizedPartitionMap underloadedNodes = filterNodes(tierMapping, 
idealPartCnt, false);
-
-        // First iterate over underloaded nodes.
-        assignPendingToUnderloaded(tier, pendingMap, fullMap, 
underloadedNodes, topSnapshot, false);
-
-        if (!pending.isEmpty() && !underloadedNodes.isEmpty()) {
-            // Same, forcing updates.
-            assignPendingToUnderloaded(tier, pendingMap, fullMap, 
underloadedNodes, topSnapshot, true);
-        }
-
-        if (!pending.isEmpty())
-            assignPendingToNodes(tier, pendingMap, fullMap, topSnapshot);
-
-        assert pending.isEmpty();
-
-        pendingMap.remove(tier);
-    }
-
-    /**
-     * Assigns pending partitions to underloaded nodes.
-     *
-     * @param tier Tier to assign.
-     * @param pendingMap Pending partitions per tier.
-     * @param fullMap Full assignment map to modify.
-     * @param underloadedNodes Underloaded nodes.
-     * @param topSnapshot Topology snapshot.
-     * @param force {@code True} if partitions should be moved.
-     */
-    private void assignPendingToUnderloaded(
-        int tier,
-        Map<Integer, Queue<Integer>> pendingMap,
-        FullAssignmentMap fullMap,
-        PrioritizedPartitionMap underloadedNodes,
-        Collection<ClusterNode> topSnapshot,
-        boolean force) {
-        Iterator<Integer> it = pendingMap.get(tier).iterator();
-
-        int ideal = parts / topSnapshot.size();
-
-        while (it.hasNext()) {
-            int part = it.next();
-
-            for (PartitionSet set : underloadedNodes.assignments()) {
-                ClusterNode node = set.node();
-
-                assert node != null;
-
-                if (fullMap.assign(part, tier, node, force, pendingMap)) {
-                    // We could add partition to partition map without 
forcing, remove partition from pending.
-                    it.remove();
-
-                    if (set.size() <= ideal)
-                        underloadedNodes.remove(set.nodeId());
-                    else
-                        underloadedNodes.update();
-
-                    break; // for, continue to the next partition.
-                }
-            }
-
-            if (underloadedNodes.isEmpty())
-                return;
-        }
-    }
-
-    /**
-     * Spreads pending partitions equally to all nodes in topology snapshot.
-     *
-     * @param tier Tier to assign.
-     * @param pendingMap Pending partitions per tier.
-     * @param fullMap Full assignment map to modify.
-     * @param topSnapshot Topology snapshot.
-     */
-    private void assignPendingToNodes(int tier, Map<Integer, Queue<Integer>> 
pendingMap,
-        FullAssignmentMap fullMap, List<ClusterNode> topSnapshot) {
-        Iterator<Integer> it = pendingMap.get(tier).iterator();
-
-        int idx = 0;
-
-        while (it.hasNext()) {
-            int part = it.next();
-
-            int i = idx;
-
-            boolean assigned = false;
-
-            do {
-                ClusterNode node = topSnapshot.get(i);
-
-                if (fullMap.assign(part, tier, node, false, pendingMap)) {
-                    it.remove();
-
-                    assigned = true;
-                }
-
-                i = (i + 1) % topSnapshot.size();
-
-                if (assigned)
-                    idx = i;
-            } while (i != idx);
-
-            if (!assigned) {
-                do {
-                    ClusterNode node = topSnapshot.get(i);
-
-                    if (fullMap.assign(part, tier, node, true, pendingMap)) {
-                        it.remove();
-
-                        assigned = true;
-                    }
-
-                    i = (i + 1) % topSnapshot.size();
-
-                    if (assigned)
-                        idx = i;
-                } while (i != idx);
-            }
-
-            if (!assigned)
-                throw new IllegalStateException("Failed to find assignable 
node for partition.");
-        }
-    }
-
-    /**
-     * Tries to balance assignments between existing nodes in topology.
-     *
-     * @param tier Tier to assign.
-     * @param pendingParts Pending partitions per tier.
-     * @param fullMap Full assignment map to modify.
-     * @param topSnapshot Topology snapshot.
-     */
-    private void balance(int tier, Map<Integer, Queue<Integer>> pendingParts, 
FullAssignmentMap fullMap,
-        Collection<ClusterNode> topSnapshot) {
-        int idealPartCnt = parts / topSnapshot.size();
-
-        Map<UUID, PartitionSet> mapping = fullMap.tierMapping(tier);
-
-        PrioritizedPartitionMap underloadedNodes = filterNodes(mapping, 
idealPartCnt, false);
-        PrioritizedPartitionMap overloadedNodes = filterNodes(mapping, 
idealPartCnt, true);
-
-        do {
-            boolean retry = false;
-
-            for (PartitionSet overloaded : overloadedNodes.assignments()) {
-                for (Integer part : overloaded.partitions()) {
-                    boolean assigned = false;
-
-                    for (PartitionSet underloaded : 
underloadedNodes.assignments()) {
-                        if (fullMap.assign(part, tier, underloaded.node(), 
false, pendingParts)) {
-                            // Size of partition sets has changed.
-                            if (overloaded.size() <= idealPartCnt)
-                                overloadedNodes.remove(overloaded.nodeId());
-                            else
-                                overloadedNodes.update();
-
-                            if (underloaded.size() >= idealPartCnt)
-                                underloadedNodes.remove(underloaded.nodeId());
-                            else
-                                underloadedNodes.update();
-
-                            assigned = true;
-
-                            retry = true;
-
-                            break;
-                        }
-                    }
-
-                    if (!assigned) {
-                        for (PartitionSet underloaded : 
underloadedNodes.assignments()) {
-                            if (fullMap.assign(part, tier, underloaded.node(), 
true, pendingParts)) {
-                                // Size of partition sets has changed.
-                                if (overloaded.size() <= idealPartCnt)
-                                    
overloadedNodes.remove(overloaded.nodeId());
-                                else
-                                    overloadedNodes.update();
-
-                                if (underloaded.size() >= idealPartCnt)
-                                    
underloadedNodes.remove(underloaded.nodeId());
-                                else
-                                    underloadedNodes.update();
-
-                                retry = true;
-
-                                break;
-                            }
-                        }
-                    }
-
-                    if (retry)
-                        break; // for part.
-                }
-
-                if (retry)
-                    break; // for overloaded.
-            }
-
-            if (!retry)
-                break;
-        }
-        while (true);
-    }
-
-    /**
-     * Constructs underloaded or overloaded partition map.
-     *
-     * @param mapping Mapping to filter.
-     * @param idealPartCnt Ideal number of partitions per node.
-     * @param overloaded {@code True} if should create overloaded map, {@code 
false} for underloaded.
-     * @return Prioritized partition map.
-     */
-    private PrioritizedPartitionMap filterNodes(Map<UUID, PartitionSet> 
mapping, int idealPartCnt, boolean overloaded) {
-        assert mapping != null;
-
-        PrioritizedPartitionMap res = new PrioritizedPartitionMap(overloaded ? 
DESC_CMP : ASC_CMP);
-
-        for (PartitionSet set : mapping.values()) {
-            if ((overloaded && set.size() > idealPartCnt) || (!overloaded && 
set.size() < idealPartCnt))
-               res.add(set);
-        }
-
-        return res;
-    }
-
-    /**
-     * Creates copy of previous partition assignment.
-     *
-     * @param ctx Affinity function context.
-     * @param topSnapshot Topology snapshot.
-     * @return Assignment copy and per node partition map.
-     */
-    private IgniteBiTuple<List<List<ClusterNode>>, Map<UUID, PartitionSet>> 
createCopy(
-        GridCacheAffinityFunctionContext ctx, Iterable<ClusterNode> 
topSnapshot) {
-        IgniteDiscoveryEvent discoEvt = ctx.discoveryEvent();
-
-        UUID leftNodeId = discoEvt.type() == IgniteEventType.EVT_NODE_JOINED ? 
null : discoEvt.eventNode().id();
-
-        List<List<ClusterNode>> cp = new ArrayList<>(parts);
-
-        Map<UUID, PartitionSet> parts = new HashMap<>();
-
-        for (int part = 0; part < this.parts; part++) {
-            List<ClusterNode> partNodes = ctx.previousAssignment(part);
-
-            List<ClusterNode> partNodesCp = new ArrayList<>(partNodes.size());
-
-            for (ClusterNode affNode : partNodes) {
-                if (!affNode.id().equals(leftNodeId)) {
-                    partNodesCp.add(affNode);
-
-                    PartitionSet partSet = parts.get(affNode.id());
-
-                    if (partSet == null) {
-                        partSet = new PartitionSet(affNode);
-
-                        parts.put(affNode.id(), partSet);
-                    }
-
-                    partSet.add(part);
-                }
-            }
-
-            cp.add(partNodesCp);
-        }
-
-        if (leftNodeId == null) {
-            // Node joined, find it and add empty set to mapping.
-            ClusterNode joinedNode = null;
-
-            for (ClusterNode node : topSnapshot) {
-                if (node.id().equals(discoEvt.eventNode().id())) {
-                    joinedNode = node;
-
-                    break;
-                }
-            }
-
-            assert joinedNode != null;
-
-            parts.put(joinedNode.id(), new PartitionSet(joinedNode));
-        }
-
-        return F.t(cp, parts);
-    }
-
-    /**
-     *
-     */
-    private static class PartitionSetComparator implements 
Comparator<PartitionSet>, Serializable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** */
-        private boolean descending;
-
-        /**
-         * @param descending {@code True} if comparator should be descending.
-         */
-        private PartitionSetComparator(boolean descending) {
-            this.descending = descending;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int compare(PartitionSet o1, PartitionSet o2) {
-            int res = o1.parts.size() < o2.parts.size() ? -1 : o1.parts.size() 
> o2.parts.size() ? 1 : 0;
-
-            return descending ? -res : res;
-        }
-    }
-
-    /**
-     * Prioritized partition map. Ordered structure in which nodes are ordered 
in ascending or descending order
-     * by number of partitions assigned to a node.
-     */
-    private static class PrioritizedPartitionMap {
-        /** Comparator. */
-        private Comparator<PartitionSet> cmp;
-
-        /** Assignment map. */
-        private Map<UUID, PartitionSet> assignmentMap = new HashMap<>();
-
-        /** Assignment list, ordered according to comparator. */
-        private List<PartitionSet> assignmentList = new ArrayList<>();
-
-        /**
-         * @param cmp Comparator.
-         */
-        private PrioritizedPartitionMap(Comparator<PartitionSet> cmp) {
-            this.cmp = cmp;
-        }
-
-        /**
-         * @param set Partition set to add.
-         */
-        public void add(PartitionSet set) {
-            PartitionSet old = assignmentMap.put(set.nodeId(), set);
-
-            if (old == null) {
-                assignmentList.add(set);
-
-                update();
-            }
-        }
-
-        /**
-         * Sorts assignment list.
-         */
-        public void update() {
-            Collections.sort(assignmentList, cmp);
-        }
-
-        /**
-         * @return Sorted assignment list.
-         */
-        public List<PartitionSet> assignments() {
-            return assignmentList;
-        }
-
-        public void remove(UUID uuid) {
-            PartitionSet rmv = assignmentMap.remove(uuid);
-
-            assignmentList.remove(rmv);
-        }
-
-        public boolean isEmpty() {
-            return assignmentList.isEmpty();
-        }
-    }
-
-    /**
-     * Constructs assignment map for specified tier.
-     *
-     * @param tier Tier number, -1 for all tiers altogether.
-     * @param assignment Assignment to construct map from.
-     * @param topSnapshot Topology snapshot.
-     * @return Assignment map.
-     */
-    private static Map<UUID, PartitionSet> assignments(int tier, 
List<List<ClusterNode>> assignment,
-        Collection<ClusterNode> topSnapshot) {
-        Map<UUID, PartitionSet> tmp = new LinkedHashMap<>();
-
-        for (int part = 0; part < assignment.size(); part++) {
-            List<ClusterNode> nodes = assignment.get(part);
-
-            assert nodes instanceof RandomAccess;
-
-            if (nodes.size() <= tier)
-                continue;
-
-            int start = tier < 0 ? 0 : tier;
-            int end = tier < 0 ? nodes.size() : tier + 1;
-
-            for (int i = start; i < end; i++) {
-                ClusterNode n = nodes.get(i);
-
-                PartitionSet set = tmp.get(n.id());
-
-                if (set == null) {
-                    set = new PartitionSet(n);
-
-                    tmp.put(n.id(), set);
-                }
-
-                set.add(part);
-            }
-        }
-
-        if (tmp.size() < topSnapshot.size()) {
-            for (ClusterNode node : topSnapshot) {
-                if (!tmp.containsKey(node.id()))
-                    tmp.put(node.id(), new PartitionSet(node));
-            }
-        }
-
-        return tmp;
-    }
-
-    /**
-     * Full assignment map. Auxiliary data structure which maintains resulting 
assignment and temporary
-     * maps consistent.
-     */
-    @SuppressWarnings("unchecked")
-    private static class FullAssignmentMap {
-        /** Per-tier assignment maps. */
-        private Map<UUID, PartitionSet>[] tierMaps;
-
-        /** Full assignment map. */
-        private Map<UUID, PartitionSet> fullMap;
-
-        /** Resulting assignment. */
-        private List<List<ClusterNode>> assignments;
-
-        /**
-         * @param tiers Number of tiers.
-         * @param assignments Assignments to modify.
-         * @param topSnapshot Topology snapshot.
-         */
-        private FullAssignmentMap(int tiers, List<List<ClusterNode>> 
assignments, Collection<ClusterNode> topSnapshot) {
-            this.assignments = assignments;
-
-            tierMaps = new Map[tiers];
-
-            for (int tier = 0; tier < tiers; tier++)
-                tierMaps[tier] = assignments(tier, assignments, topSnapshot);
-
-            fullMap = assignments(-1, assignments, topSnapshot);
-        }
-
-        /**
-         * Tries to assign partition to given node on specified tier. If force 
is false, assignment will succeed
-         * only if this partition is not already assigned to a node. If force 
is true, then assignment will succeed
-         * only if partition is not assigned to a tier with number less than 
passed in. Assigned partition from
-         * greater tier will be moved to pending queue.
-         *
-         * @param part Partition to assign.
-         * @param tier Tier number to assign.
-         * @param node Node to move partition to.
-         * @param force Force flag.
-         * @param pendingParts per tier pending partitions map.
-         * @return {@code True} if assignment succeeded.
-         */
-        boolean assign(int part, int tier, ClusterNode node, boolean force, 
Map<Integer, Queue<Integer>> pendingParts) {
-            UUID nodeId = node.id();
-
-            if (!fullMap.get(nodeId).contains(part)) {
-                tierMaps[tier].get(nodeId).add(part);
-
-                fullMap.get(nodeId).add(part);
-
-                List<ClusterNode> assignment = assignments.get(part);
-
-                if (assignment.size() <= tier)
-                    assignment.add(node);
-                else {
-                    ClusterNode oldNode = assignment.set(tier, node);
-
-                    if (oldNode != null) {
-                        UUID oldNodeId = oldNode.id();
-
-                        tierMaps[tier].get(oldNodeId).remove(part);
-                        fullMap.get(oldNodeId).remove(part);
-                    }
-                }
-
-                return true;
-            }
-            else if (force) {
-                assert !tierMaps[tier].get(nodeId).contains(part);
-
-                // Check previous tiers first.
-                for (int t = 0; t < tier; t++) {
-                    if (tierMaps[t].get(nodeId).contains(part))
-                        return false;
-                }
-
-                // Partition is on some lower tier, switch it.
-                for (int t = tier + 1; t < tierMaps.length; t++) {
-                    if (tierMaps[t].get(nodeId).contains(part)) {
-                        ClusterNode oldNode = assignments.get(part).get(tier);
-
-                        // Move partition from level t to tier.
-                        assignments.get(part).set(tier, node);
-                        assignments.get(part).set(t, null);
-
-                        if (oldNode != null) {
-                            tierMaps[tier].get(oldNode.id()).remove(part);
-                            fullMap.get(oldNode.id()).remove(part);
-                        }
-
-                        tierMaps[tier].get(nodeId).add(part);
-                        tierMaps[t].get(nodeId).remove(part);
-
-                        Queue<Integer> pending = pendingParts.get(t);
-
-                        if (pending == null) {
-                            pending = new LinkedList<>();
-
-                            pendingParts.put(t, pending);
-                        }
-
-                        pending.add(part);
-
-                        return true;
-                    }
-                }
-
-                throw new IllegalStateException("Unable to assign partition to 
node while force is true.");
-            }
-
-            // !force.
-            return false;
-        }
-
-        /**
-         * Gets tier mapping.
-         *
-         * @param tier Tier to get mapping.
-         * @return Per node map.
-         */
-        public Map<UUID, PartitionSet> tierMapping(int tier) {
-            return tierMaps[tier];
-        }
-    }
-
-    /**
-     * Applies a supplemental hash function to a given hashCode, which
-     * defends against poor quality hash functions.
-     *
-     * @param h Hash code.
-     * @return Enhanced hash code.
-     */
-    private static int hash(int h) {
-        // Spread bits to regularize both segment and index locations,
-        // using variant of single-word Wang/Jenkins hash.
-        h += (h <<  15) ^ 0xffffcd7d;
-        h ^= (h >>> 10);
-        h += (h <<   3);
-        h ^= (h >>>  6);
-        h += (h <<   2) + (h << 14);
-        return h ^ (h >>> 16);
-    }
-
-    @SuppressWarnings("ComparableImplementedButEqualsNotOverridden")
-    private static class PartitionSet {
-        /** */
-        private ClusterNode node;
-
-        /** Partitions. */
-        private Collection<Integer> parts = new LinkedList<>();
-
-        /**
-         * @param node Node.
-         */
-        private PartitionSet(ClusterNode node) {
-            this.node = node;
-        }
-
-        /**
-         * @return Node.
-         */
-        private ClusterNode node() {
-            return node;
-        }
-
-        /**
-         * @return Node ID.
-         */
-        private UUID nodeId() {
-            return node.id();
-        }
-
-        /**
-         * @return Partition set size.
-         */
-        private int size() {
-            return parts.size();
-        }
-
-        /**
-         * Adds partition to partition set.
-         *
-         * @param part Partition to add.
-         * @return {@code True} if partition was added, {@code false} if 
partition already exists.
-         */
-        private boolean add(int part) {
-            if (!parts.contains(part)) {
-                parts.add(part);
-
-                return true;
-            }
-
-            return false;
-        }
-
-        /**
-         * @param part Partition to remove.
-         */
-        private void remove(Integer part) {
-            parts.remove(part); // Remove object, not index.
-        }
-
-        /**
-         * @return Partitions.
-         */
-        @SuppressWarnings("TypeMayBeWeakened")
-        private Collection<Integer> partitions() {
-            return parts;
-        }
-
-        /**
-         * Checks if partition set contains given partition.
-         *
-         * @param part Partition to check.
-         * @return {@code True} if partition set contains given partition.
-         */
-        private boolean contains(int part) {
-            return parts.contains(part);
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return "PartSet [nodeId=" + node.id() + ", size=" + parts.size() + 
", parts=" + parts + ']';
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/CacheRendezvousAffinityFunction.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/CacheRendezvousAffinityFunction.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/CacheRendezvousAffinityFunction.java
new file mode 100644
index 0000000..ffb6187
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/CacheRendezvousAffinityFunction.java
@@ -0,0 +1,502 @@
+/*
+ * 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.rendezvous;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.marshaller.*;
+import org.apache.ignite.marshaller.optimized.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.security.*;
+import java.util.*;
+
+/**
+ * Affinity function for partitioned cache based on Highest Random Weight 
algorithm.
+ * This function supports the following configuration:
+ * <ul>
+ * <li>
+ *      {@code partitions} - Number of partitions to spread across nodes.
+ * </li>
+ * <li>
+ *      {@code excludeNeighbors} - If set to {@code true}, will exclude 
same-host-neighbors
+ *      from being backups of each other. Note that {@code backupFilter} is 
ignored if
+ *      {@code excludeNeighbors} is set to {@code true}.
+ * </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 
CacheConfiguration#getAffinity()} method.
+ */
+public class CacheRendezvousAffinityFunction implements CacheAffinityFunction, 
Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Default number of partitions. */
+    public static final int DFLT_PARTITION_COUNT = 10000;
+
+    /** Comparator. */
+    private static final Comparator<IgniteBiTuple<Long, ClusterNode>> 
COMPARATOR =
+        new HashComparator();
+
+    /** Thread local message digest. */
+    private ThreadLocal<MessageDigest> digest = new 
ThreadLocal<MessageDigest>() {
+        @Override protected MessageDigest initialValue() {
+            try {
+                return MessageDigest.getInstance("MD5");
+            }
+            catch (NoSuchAlgorithmException e) {
+                assert false : "Should have failed in constructor";
+
+                throw new IgniteException("Failed to obtain message digest 
(digest was available in constructor)",
+                    e);
+            }
+        }
+    };
+
+    /** Number of partitions. */
+    private int parts;
+
+    /** Exclude neighbors flag. */
+    private boolean exclNeighbors;
+
+    /** Optional backup filter. First node is primary, second node is a node 
being tested. */
+    private IgniteBiPredicate<ClusterNode, ClusterNode> backupFilter;
+
+    /** Hash ID resolver. */
+    private CacheAffinityNodeHashResolver hashIdRslvr = new 
CacheAffinityNodeAddressHashResolver();
+
+    /** Marshaller. */
+    private IgniteMarshaller marshaller = new IgniteOptimizedMarshaller(false);
+
+    /**
+     * Empty constructor with all defaults.
+     */
+    public CacheRendezvousAffinityFunction() {
+        this(false);
+    }
+
+    /**
+     * 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 CacheRendezvousAffinityFunction(boolean exclNeighbors) {
+        this(exclNeighbors, DFLT_PARTITION_COUNT);
+    }
+
+    /**
+     * 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 CacheRendezvousAffinityFunction(boolean exclNeighbors, int parts) {
+        this(exclNeighbors, parts, null);
+    }
+
+    /**
+     * 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 CacheRendezvousAffinityFunction(int parts,
+                                           @Nullable 
IgniteBiPredicate<ClusterNode, ClusterNode> backupFilter) {
+        this(false, parts, backupFilter);
+    }
+
+    /**
+     * Private constructor.
+     *
+     * @param exclNeighbors Exclude neighbors flag.
+     * @param parts Partitions count.
+     * @param backupFilter Backup filter.
+     */
+    private CacheRendezvousAffinityFunction(boolean exclNeighbors, int parts,
+                                            IgniteBiPredicate<ClusterNode, 
ClusterNode> backupFilter) {
+        A.ensure(parts != 0, "parts != 0");
+
+        this.exclNeighbors = exclNeighbors;
+        this.parts = parts;
+        this.backupFilter = backupFilter;
+
+        try {
+            MessageDigest.getInstance("MD5");
+        }
+        catch (NoSuchAlgorithmException e) {
+            throw new IgniteException("Failed to obtain MD5 message digest 
instance.", e);
+        }
+    }
+
+    /**
+     * 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;
+    }
+
+    /**
+     * 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;
+    }
+
+    /**
+     * Returns collection of nodes (primary first) for specified partition.
+     */
+    public List<ClusterNode> assignPartition(int part, List<ClusterNode> 
nodes, int backups,
+        @Nullable Map<UUID, Collection<ClusterNode>> neighborhoodCache) {
+        if (nodes.size() <= 1)
+            return nodes;
+
+        List<IgniteBiTuple<Long, ClusterNode>> lst = new ArrayList<>();
+
+        MessageDigest d = digest.get();
+
+        for (ClusterNode node : nodes) {
+            Object nodeHash = hashIdRslvr.resolve(node);
+
+            try {
+                ByteArrayOutputStream out = new ByteArrayOutputStream();
+
+                byte[] nodeHashBytes = marshaller.marshal(nodeHash);
+
+                out.write(nodeHashBytes, 0, nodeHashBytes.length); // Avoid 
IOException.
+                out.write(U.intToBytes(part), 0, 4); // Avoid IOException.
+
+                d.reset();
+
+                byte[] bytes = d.digest(out.toByteArray());
+
+                long hash =
+                      (bytes[0] & 0xFFL)
+                    | ((bytes[1] & 0xFFL) << 8)
+                    | ((bytes[2] & 0xFFL) << 16)
+                    | ((bytes[3] & 0xFFL) << 24)
+                    | ((bytes[4] & 0xFFL) << 32)
+                    | ((bytes[5] & 0xFFL) << 40)
+                    | ((bytes[6] & 0xFFL) << 48)
+                    | ((bytes[7] & 0xFFL) << 56);
+
+                lst.add(F.t(hash, node));
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException(e);
+            }
+        }
+
+        Collections.sort(lst, COMPARATOR);
+
+        int primaryAndBackups;
+
+        List<ClusterNode> res;
+
+        if (backups == Integer.MAX_VALUE) {
+            primaryAndBackups = Integer.MAX_VALUE;
+
+            res = new ArrayList<>();
+        }
+        else {
+            primaryAndBackups = backups + 1;
+
+            res = new ArrayList<>(primaryAndBackups);
+        }
+
+        ClusterNode primary = lst.get(0).get2();
+
+        res.add(primary);
+
+        // Select backups.
+        if (backups > 0) {
+            for (int i = 1; i < lst.size(); i++) {
+                IgniteBiTuple<Long, ClusterNode> next = lst.get(i);
+
+                ClusterNode node = next.get2();
+
+                if (exclNeighbors) {
+                    Collection<ClusterNode> allNeighbors = 
allNeighbors(neighborhoodCache, res);
+
+                    if (!allNeighbors.contains(node))
+                        res.add(node);
+                }
+                else {
+                    if (!res.contains(node) && (backupFilter == null || 
backupFilter.apply(primary, node)))
+                        res.add(next.get2());
+                }
+
+                if (res.size() == primaryAndBackups)
+                    break;
+            }
+        }
+
+        if (res.size() < primaryAndBackups && nodes.size() >= 
primaryAndBackups && exclNeighbors) {
+            // Need to iterate one more time in case if there are no nodes 
which pass exclude backups criteria.
+            for (int i = 1; i < lst.size(); i++) {
+                IgniteBiTuple<Long, ClusterNode> next = lst.get(i);
+
+                ClusterNode node = next.get2();
+
+                if (!res.contains(node))
+                    res.add(next.get2());
+
+                if (res.size() == primaryAndBackups)
+                    break;
+            }
+        }
+
+        assert res.size() <= primaryAndBackups;
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void reset() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public int partitions() {
+        return parts;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int partition(Object key) {
+        return U.safeAbs(key.hashCode() % parts);
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<List<ClusterNode>> 
assignPartitions(CacheAffinityFunctionContext affCtx) {
+        List<List<ClusterNode>> assignments = new ArrayList<>(parts);
+
+        Map<UUID, Collection<ClusterNode>> neighborhoodCache = exclNeighbors ?
+            neighbors(affCtx.currentTopologySnapshot()) : null;
+
+        for (int i = 0; i < parts; i++) {
+            List<ClusterNode> partAssignment = assignPartition(i, 
affCtx.currentTopologySnapshot(), affCtx.backups(),
+                neighborhoodCache);
+
+            assignments.add(partAssignment);
+        }
+
+        return assignments;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeNode(UUID nodeId) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeInt(parts);
+        out.writeBoolean(exclNeighbors);
+        out.writeObject(hashIdRslvr);
+        out.writeObject(backupFilter);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, 
ClassNotFoundException {
+        parts = in.readInt();
+        exclNeighbors = in.readBoolean();
+        hashIdRslvr = (CacheAffinityNodeHashResolver)in.readObject();
+        backupFilter = (IgniteBiPredicate<ClusterNode, 
ClusterNode>)in.readObject();
+    }
+
+    /**
+     * Builds neighborhood map for all nodes in snapshot.
+     *
+     * @param topSnapshot Topology snapshot.
+     * @return Neighbors map.
+     */
+    private Map<UUID, Collection<ClusterNode>> 
neighbors(Collection<ClusterNode> topSnapshot) {
+        Map<String, Collection<ClusterNode>> macMap = new 
HashMap<>(topSnapshot.size(), 1.0f);
+
+        // Group by mac addresses.
+        for (ClusterNode node : topSnapshot) {
+            String macs = node.attribute(GridNodeAttributes.ATTR_MACS);
+
+            Collection<ClusterNode> nodes = macMap.get(macs);
+
+            if (nodes == null) {
+                nodes = new HashSet<>();
+
+                macMap.put(macs, nodes);
+            }
+
+            nodes.add(node);
+        }
+
+        Map<UUID, Collection<ClusterNode>> neighbors = new 
HashMap<>(topSnapshot.size(), 1.0f);
+
+        for (Collection<ClusterNode> group : macMap.values()) {
+            for (ClusterNode node : group)
+                neighbors.put(node.id(), group);
+        }
+
+        return neighbors;
+    }
+
+    /**
+     * @param neighborhoodCache Neighborhood cache.
+     * @param nodes Nodes.
+     * @return All neighbors for given nodes.
+     */
+    private Collection<ClusterNode> allNeighbors(Map<UUID, 
Collection<ClusterNode>> neighborhoodCache,
+        Iterable<ClusterNode> nodes) {
+        Collection<ClusterNode> res = new HashSet<>();
+
+        for (ClusterNode node : nodes) {
+            if (!res.contains(node))
+                res.addAll(neighborhoodCache.get(node.id()));
+        }
+
+        return res;
+    }
+
+    /**
+     *
+     */
+    private static class HashComparator implements 
Comparator<IgniteBiTuple<Long, ClusterNode>>, Serializable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public int compare(IgniteBiTuple<Long, ClusterNode> o1, 
IgniteBiTuple<Long, ClusterNode> o2) {
+            return o1.get1() < o2.get1() ? -1 : o1.get1() > o2.get1() ? 1 :
+                o1.get2().id().compareTo(o2.get2().id());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/GridCacheRendezvousAffinityFunction.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/GridCacheRendezvousAffinityFunction.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/GridCacheRendezvousAffinityFunction.java
deleted file mode 100644
index ce2429b..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/GridCacheRendezvousAffinityFunction.java
+++ /dev/null
@@ -1,502 +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.rendezvous;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.marshaller.*;
-import org.apache.ignite.marshaller.optimized.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.security.*;
-import java.util.*;
-
-/**
- * Affinity function for partitioned cache based on Highest Random Weight 
algorithm.
- * This function supports the following configuration:
- * <ul>
- * <li>
- *      {@code partitions} - Number of partitions to spread across nodes.
- * </li>
- * <li>
- *      {@code excludeNeighbors} - If set to {@code true}, will exclude 
same-host-neighbors
- *      from being backups of each other. Note that {@code backupFilter} is 
ignored if
- *      {@code excludeNeighbors} is set to {@code true}.
- * </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 
CacheConfiguration#getAffinity()} method.
- */
-public class GridCacheRendezvousAffinityFunction implements 
GridCacheAffinityFunction, Externalizable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Default number of partitions. */
-    public static final int DFLT_PARTITION_COUNT = 10000;
-
-    /** Comparator. */
-    private static final Comparator<IgniteBiTuple<Long, ClusterNode>> 
COMPARATOR =
-        new HashComparator();
-
-    /** Thread local message digest. */
-    private ThreadLocal<MessageDigest> digest = new 
ThreadLocal<MessageDigest>() {
-        @Override protected MessageDigest initialValue() {
-            try {
-                return MessageDigest.getInstance("MD5");
-            }
-            catch (NoSuchAlgorithmException e) {
-                assert false : "Should have failed in constructor";
-
-                throw new IgniteException("Failed to obtain message digest 
(digest was available in constructor)",
-                    e);
-            }
-        }
-    };
-
-    /** Number of partitions. */
-    private int parts;
-
-    /** Exclude neighbors flag. */
-    private boolean exclNeighbors;
-
-    /** Optional backup filter. First node is primary, second node is a node 
being tested. */
-    private IgniteBiPredicate<ClusterNode, ClusterNode> backupFilter;
-
-    /** Hash ID resolver. */
-    private GridCacheAffinityNodeHashResolver hashIdRslvr = new 
GridCacheAffinityNodeAddressHashResolver();
-
-    /** Marshaller. */
-    private IgniteMarshaller marshaller = new IgniteOptimizedMarshaller(false);
-
-    /**
-     * Empty constructor with all defaults.
-     */
-    public GridCacheRendezvousAffinityFunction() {
-        this(false);
-    }
-
-    /**
-     * 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 GridCacheRendezvousAffinityFunction(boolean exclNeighbors) {
-        this(exclNeighbors, DFLT_PARTITION_COUNT);
-    }
-
-    /**
-     * 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 GridCacheRendezvousAffinityFunction(boolean exclNeighbors, int 
parts) {
-        this(exclNeighbors, parts, null);
-    }
-
-    /**
-     * 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 GridCacheRendezvousAffinityFunction(int parts,
-        @Nullable IgniteBiPredicate<ClusterNode, ClusterNode> backupFilter) {
-        this(false, parts, backupFilter);
-    }
-
-    /**
-     * Private constructor.
-     *
-     * @param exclNeighbors Exclude neighbors flag.
-     * @param parts Partitions count.
-     * @param backupFilter Backup filter.
-     */
-    private GridCacheRendezvousAffinityFunction(boolean exclNeighbors, int 
parts,
-        IgniteBiPredicate<ClusterNode, ClusterNode> backupFilter) {
-        A.ensure(parts != 0, "parts != 0");
-
-        this.exclNeighbors = exclNeighbors;
-        this.parts = parts;
-        this.backupFilter = backupFilter;
-
-        try {
-            MessageDigest.getInstance("MD5");
-        }
-        catch (NoSuchAlgorithmException e) {
-            throw new IgniteException("Failed to obtain MD5 message digest 
instance.", e);
-        }
-    }
-
-    /**
-     * 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 GridCacheAffinityNodeHashResolver 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(GridCacheAffinityNodeHashResolver 
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;
-    }
-
-    /**
-     * 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;
-    }
-
-    /**
-     * Returns collection of nodes (primary first) for specified partition.
-     */
-    public List<ClusterNode> assignPartition(int part, List<ClusterNode> 
nodes, int backups,
-        @Nullable Map<UUID, Collection<ClusterNode>> neighborhoodCache) {
-        if (nodes.size() <= 1)
-            return nodes;
-
-        List<IgniteBiTuple<Long, ClusterNode>> lst = new ArrayList<>();
-
-        MessageDigest d = digest.get();
-
-        for (ClusterNode node : nodes) {
-            Object nodeHash = hashIdRslvr.resolve(node);
-
-            try {
-                ByteArrayOutputStream out = new ByteArrayOutputStream();
-
-                byte[] nodeHashBytes = marshaller.marshal(nodeHash);
-
-                out.write(nodeHashBytes, 0, nodeHashBytes.length); // Avoid 
IOException.
-                out.write(U.intToBytes(part), 0, 4); // Avoid IOException.
-
-                d.reset();
-
-                byte[] bytes = d.digest(out.toByteArray());
-
-                long hash =
-                      (bytes[0] & 0xFFL)
-                    | ((bytes[1] & 0xFFL) << 8)
-                    | ((bytes[2] & 0xFFL) << 16)
-                    | ((bytes[3] & 0xFFL) << 24)
-                    | ((bytes[4] & 0xFFL) << 32)
-                    | ((bytes[5] & 0xFFL) << 40)
-                    | ((bytes[6] & 0xFFL) << 48)
-                    | ((bytes[7] & 0xFFL) << 56);
-
-                lst.add(F.t(hash, node));
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException(e);
-            }
-        }
-
-        Collections.sort(lst, COMPARATOR);
-
-        int primaryAndBackups;
-
-        List<ClusterNode> res;
-
-        if (backups == Integer.MAX_VALUE) {
-            primaryAndBackups = Integer.MAX_VALUE;
-
-            res = new ArrayList<>();
-        }
-        else {
-            primaryAndBackups = backups + 1;
-
-            res = new ArrayList<>(primaryAndBackups);
-        }
-
-        ClusterNode primary = lst.get(0).get2();
-
-        res.add(primary);
-
-        // Select backups.
-        if (backups > 0) {
-            for (int i = 1; i < lst.size(); i++) {
-                IgniteBiTuple<Long, ClusterNode> next = lst.get(i);
-
-                ClusterNode node = next.get2();
-
-                if (exclNeighbors) {
-                    Collection<ClusterNode> allNeighbors = 
allNeighbors(neighborhoodCache, res);
-
-                    if (!allNeighbors.contains(node))
-                        res.add(node);
-                }
-                else {
-                    if (!res.contains(node) && (backupFilter == null || 
backupFilter.apply(primary, node)))
-                        res.add(next.get2());
-                }
-
-                if (res.size() == primaryAndBackups)
-                    break;
-            }
-        }
-
-        if (res.size() < primaryAndBackups && nodes.size() >= 
primaryAndBackups && exclNeighbors) {
-            // Need to iterate one more time in case if there are no nodes 
which pass exclude backups criteria.
-            for (int i = 1; i < lst.size(); i++) {
-                IgniteBiTuple<Long, ClusterNode> next = lst.get(i);
-
-                ClusterNode node = next.get2();
-
-                if (!res.contains(node))
-                    res.add(next.get2());
-
-                if (res.size() == primaryAndBackups)
-                    break;
-            }
-        }
-
-        assert res.size() <= primaryAndBackups;
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void reset() {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public int partitions() {
-        return parts;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int partition(Object key) {
-        return U.safeAbs(key.hashCode() % parts);
-    }
-
-    /** {@inheritDoc} */
-    @Override public List<List<ClusterNode>> 
assignPartitions(GridCacheAffinityFunctionContext affCtx) {
-        List<List<ClusterNode>> assignments = new ArrayList<>(parts);
-
-        Map<UUID, Collection<ClusterNode>> neighborhoodCache = exclNeighbors ?
-            neighbors(affCtx.currentTopologySnapshot()) : null;
-
-        for (int i = 0; i < parts; i++) {
-            List<ClusterNode> partAssignment = assignPartition(i, 
affCtx.currentTopologySnapshot(), affCtx.backups(),
-                neighborhoodCache);
-
-            assignments.add(partAssignment);
-        }
-
-        return assignments;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeNode(UUID nodeId) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeInt(parts);
-        out.writeBoolean(exclNeighbors);
-        out.writeObject(hashIdRslvr);
-        out.writeObject(backupFilter);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, 
ClassNotFoundException {
-        parts = in.readInt();
-        exclNeighbors = in.readBoolean();
-        hashIdRslvr = (GridCacheAffinityNodeHashResolver)in.readObject();
-        backupFilter = (IgniteBiPredicate<ClusterNode, 
ClusterNode>)in.readObject();
-    }
-
-    /**
-     * Builds neighborhood map for all nodes in snapshot.
-     *
-     * @param topSnapshot Topology snapshot.
-     * @return Neighbors map.
-     */
-    private Map<UUID, Collection<ClusterNode>> 
neighbors(Collection<ClusterNode> topSnapshot) {
-        Map<String, Collection<ClusterNode>> macMap = new 
HashMap<>(topSnapshot.size(), 1.0f);
-
-        // Group by mac addresses.
-        for (ClusterNode node : topSnapshot) {
-            String macs = node.attribute(GridNodeAttributes.ATTR_MACS);
-
-            Collection<ClusterNode> nodes = macMap.get(macs);
-
-            if (nodes == null) {
-                nodes = new HashSet<>();
-
-                macMap.put(macs, nodes);
-            }
-
-            nodes.add(node);
-        }
-
-        Map<UUID, Collection<ClusterNode>> neighbors = new 
HashMap<>(topSnapshot.size(), 1.0f);
-
-        for (Collection<ClusterNode> group : macMap.values()) {
-            for (ClusterNode node : group)
-                neighbors.put(node.id(), group);
-        }
-
-        return neighbors;
-    }
-
-    /**
-     * @param neighborhoodCache Neighborhood cache.
-     * @param nodes Nodes.
-     * @return All neighbors for given nodes.
-     */
-    private Collection<ClusterNode> allNeighbors(Map<UUID, 
Collection<ClusterNode>> neighborhoodCache,
-        Iterable<ClusterNode> nodes) {
-        Collection<ClusterNode> res = new HashSet<>();
-
-        for (ClusterNode node : nodes) {
-            if (!res.contains(node))
-                res.addAll(neighborhoodCache.get(node.id()));
-        }
-
-        return res;
-    }
-
-    /**
-     *
-     */
-    private static class HashComparator implements 
Comparator<IgniteBiTuple<Long, ClusterNode>>, Serializable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** {@inheritDoc} */
-        @Override public int compare(IgniteBiTuple<Long, ClusterNode> o1, 
IgniteBiTuple<Long, ClusterNode> o2) {
-            return o1.get1() < o2.get1() ? -1 : o1.get1() > o2.get1() ? 1 :
-                o1.get2().id().compareTo(o2.get2().id());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/cloner/CacheBasicCloner.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/cloner/CacheBasicCloner.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/cloner/CacheBasicCloner.java
new file mode 100644
index 0000000..7a31d2c
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/cache/cloner/CacheBasicCloner.java
@@ -0,0 +1,34 @@
+/*
+ * 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.cloner;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.util.typedef.*;
+
+/**
+ * Basic cache cloner based on utilization of {@link Cloneable} interface. If
+ * a passed in object implements {@link Cloneable} then its implementation of
+ * {@link Object#clone()} is used to get a copy. Otherwise, the object itself
+ * will be returned without cloning.
+ */
+public class CacheBasicCloner implements CacheCloner {
+    /** {@inheritDoc} */
+    @Override public <T> T cloneValue(T val) throws IgniteCheckedException {
+        return X.cloneObject(val, false, true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/cloner/CacheCloner.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/cloner/CacheCloner.java 
b/modules/core/src/main/java/org/apache/ignite/cache/cloner/CacheCloner.java
new file mode 100644
index 0000000..91b6264
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/cloner/CacheCloner.java
@@ -0,0 +1,58 @@
+/*
+ * 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.cloner;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.jetbrains.annotations.*;
+
+/**
+ * Cache cloner which clones cached values before returning them from cache.
+ * It will only be used if {@link org.apache.ignite.cache.CacheFlag#CLONE} 
flag is set
+ * on projection which the user is working with (this flag is disabled
+ * by default).
+ * <p>
+ * This behaviour is useful, as a an example, when we need to get some object
+ * from cache, change some of its properties and put it back into cache.
+ * In such a scenario it would be wrong to change properties of cached value
+ * itself without creating a copy first, since it would break cache integrity,
+ * and will affect the cached values returned to other threads even before
+ * the transaction commits.
+ * <p>
+ * Cache cloner can be set in cache configuration via {@link 
CacheConfiguration#getCloner()}
+ * method. By default, cache uses {@link CacheBasicCloner} implementation
+ * which will clone only objects implementing {@link Cloneable} interface. You
+ * can also configure cache to use {@link CacheDeepCloner} which will perform
+ * deep-cloning of all objects returned from cache, regardless of the
+ * {@link Cloneable} interface. If none of the above cloners fit your logic, 
you
+ * can also provide your own implementation of this interface.
+ *
+ * @see CacheBasicCloner
+ * @see CacheDeepCloner
+ * @see CacheConfiguration#getCloner()
+ * @see CacheConfiguration#setCloner(CacheCloner)
+ *
+ */
+public interface CacheCloner {
+    /**
+     * @param val Object to make a clone for.
+     * @throws IgniteCheckedException If failed to clone given object.
+     * @return Clone for given object.
+     */
+    @Nullable public <T> T cloneValue(T val) throws IgniteCheckedException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/cloner/CacheDeepCloner.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/cloner/CacheDeepCloner.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/cloner/CacheDeepCloner.java
new file mode 100644
index 0000000..84508b8
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/cache/cloner/CacheDeepCloner.java
@@ -0,0 +1,87 @@
+/*
+ * 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.cloner;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.util.typedef.*;
+
+/**
+ * Cache deep cloner that creates a copy of an object using deep reflection.
+ * <p>
+ * If {@link #isHonorCloneable()} is set to {@code true}, then deep cloner will
+ * first check if the passed in object implements {@link Cloneable} interface 
and,
+ * if it does, will delegate to {@code clone()} method. It is advisable for
+ * instances that need to be cloned to implement {@link Cloneable}, as cloning
+ * this way will generally be faster than reflection-based cloning.
+ * <p>
+ * This implementation will first check if the object to clone has an empty
+ * constructor. If it does, then it will be instantiated using such 
constructor.
+ * Otherwise an empty constructor will be fetched from JDK and used instead.
+ * Note that this behavior may not work on some JDKs in which case
+ * {@link #cloneValue(Object)} method will result in {@link 
IgniteCheckedException}
+ * being thrown.
+ */
+public class CacheDeepCloner implements CacheCloner {
+    /** */
+    private boolean honorCloneable = true;
+
+    /**
+     * Creates deep cloner with {@link #isHonorCloneable()} flag set to {@code 
true}.
+     */
+    public CacheDeepCloner() {
+        // No-op.
+    }
+
+    /**
+     * Creates a new instance of deep cloner with specified flag to honor
+     * {@link Cloneable} interface or not.
+     *
+     * @param honorCloneable Flag indicating whether {@link Cloneable}
+     *      interface should be honored or not when cloning.
+     */
+    public CacheDeepCloner(boolean honorCloneable) {
+        this.honorCloneable = honorCloneable;
+    }
+
+    /**
+     * Gets flag indicating if {@link Cloneable} interface should be honored
+     * when cloning, or if reflection-based deep cloning should always be 
performed.
+     *
+     * @return Flag indicating if {@link Cloneable} interface should be honored
+     *      when cloning
+     */
+    public boolean isHonorCloneable() {
+        return honorCloneable;
+    }
+
+    /**
+     * Sets flag indicating if {@link Cloneable} interface should be honored
+     * when cloning, or if reflection-based deep cloning should always be 
performed.
+     *
+     * @param honorCloneable Flag indicating whether {@link Cloneable} 
interface
+     *      should be honored or not when cloning.
+     */
+    public void setHonorCloneable(boolean honorCloneable) {
+        this.honorCloneable = honorCloneable;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T cloneValue(T val) throws IgniteCheckedException {
+        return X.cloneObject(val, true, honorCloneable);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheBasicCloner.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheBasicCloner.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheBasicCloner.java
deleted file mode 100644
index 92cd0f4..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheBasicCloner.java
+++ /dev/null
@@ -1,34 +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.cloner;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.util.typedef.*;
-
-/**
- * Basic cache cloner based on utilization of {@link Cloneable} interface. If
- * a passed in object implements {@link Cloneable} then its implementation of
- * {@link Object#clone()} is used to get a copy. Otherwise, the object itself
- * will be returned without cloning.
- */
-public class GridCacheBasicCloner implements GridCacheCloner {
-    /** {@inheritDoc} */
-    @Override public <T> T cloneValue(T val) throws IgniteCheckedException {
-        return X.cloneObject(val, false, true);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheCloner.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheCloner.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheCloner.java
deleted file mode 100644
index 892ee71..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheCloner.java
+++ /dev/null
@@ -1,58 +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.cloner;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.jetbrains.annotations.*;
-
-/**
- * Cache cloner which clones cached values before returning them from cache.
- * It will only be used if {@link org.apache.ignite.cache.GridCacheFlag#CLONE} 
flag is set
- * on projection which the user is working with (this flag is disabled
- * by default).
- * <p>
- * This behaviour is useful, as a an example, when we need to get some object
- * from cache, change some of its properties and put it back into cache.
- * In such a scenario it would be wrong to change properties of cached value
- * itself without creating a copy first, since it would break cache integrity,
- * and will affect the cached values returned to other threads even before
- * the transaction commits.
- * <p>
- * Cache cloner can be set in cache configuration via {@link 
CacheConfiguration#getCloner()}
- * method. By default, cache uses {@link GridCacheBasicCloner} implementation
- * which will clone only objects implementing {@link Cloneable} interface. You
- * can also configure cache to use {@link GridCacheDeepCloner} which will 
perform
- * deep-cloning of all objects returned from cache, regardless of the
- * {@link Cloneable} interface. If none of the above cloners fit your logic, 
you
- * can also provide your own implementation of this interface.
- *
- * @see GridCacheBasicCloner
- * @see GridCacheDeepCloner
- * @see CacheConfiguration#getCloner()
- * @see CacheConfiguration#setCloner(GridCacheCloner)
- *
- */
-public interface GridCacheCloner {
-    /**
-     * @param val Object to make a clone for.
-     * @throws IgniteCheckedException If failed to clone given object.
-     * @return Clone for given object.
-     */
-    @Nullable public <T> T cloneValue(T val) throws IgniteCheckedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheDeepCloner.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheDeepCloner.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheDeepCloner.java
deleted file mode 100644
index 5b8eff8..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheDeepCloner.java
+++ /dev/null
@@ -1,87 +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.cloner;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.util.typedef.*;
-
-/**
- * Cache deep cloner that creates a copy of an object using deep reflection.
- * <p>
- * If {@link #isHonorCloneable()} is set to {@code true}, then deep cloner will
- * first check if the passed in object implements {@link Cloneable} interface 
and,
- * if it does, will delegate to {@code clone()} method. It is advisable for
- * instances that need to be cloned to implement {@link Cloneable}, as cloning
- * this way will generally be faster than reflection-based cloning.
- * <p>
- * This implementation will first check if the object to clone has an empty
- * constructor. If it does, then it will be instantiated using such 
constructor.
- * Otherwise an empty constructor will be fetched from JDK and used instead.
- * Note that this behavior may not work on some JDKs in which case
- * {@link #cloneValue(Object)} method will result in {@link 
IgniteCheckedException}
- * being thrown.
- */
-public class GridCacheDeepCloner implements GridCacheCloner {
-    /** */
-    private boolean honorCloneable = true;
-
-    /**
-     * Creates deep cloner with {@link #isHonorCloneable()} flag set to {@code 
true}.
-     */
-    public GridCacheDeepCloner() {
-        // No-op.
-    }
-
-    /**
-     * Creates a new instance of deep cloner with specified flag to honor
-     * {@link Cloneable} interface or not.
-     *
-     * @param honorCloneable Flag indicating whether {@link Cloneable}
-     *      interface should be honored or not when cloning.
-     */
-    public GridCacheDeepCloner(boolean honorCloneable) {
-        this.honorCloneable = honorCloneable;
-    }
-
-    /**
-     * Gets flag indicating if {@link Cloneable} interface should be honored
-     * when cloning, or if reflection-based deep cloning should always be 
performed.
-     *
-     * @return Flag indicating if {@link Cloneable} interface should be honored
-     *      when cloning
-     */
-    public boolean isHonorCloneable() {
-        return honorCloneable;
-    }
-
-    /**
-     * Sets flag indicating if {@link Cloneable} interface should be honored
-     * when cloning, or if reflection-based deep cloning should always be 
performed.
-     *
-     * @param honorCloneable Flag indicating whether {@link Cloneable} 
interface
-     *      should be honored or not when cloning.
-     */
-    public void setHonorCloneable(boolean honorCloneable) {
-        this.honorCloneable = honorCloneable;
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> T cloneValue(T val) throws IgniteCheckedException {
-        return X.cloneObject(val, true, honorCloneable);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/cache/datastructures/CacheAtomicLong.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/datastructures/CacheAtomicLong.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/datastructures/CacheAtomicLong.java
new file mode 100644
index 0000000..6137827
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/cache/datastructures/CacheAtomicLong.java
@@ -0,0 +1,157 @@
+/*
+ * 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.datastructures;
+
+import org.apache.ignite.*;
+
+/**
+ * This interface provides a rich API for working with distributedly cached 
atomic long value.
+ * <p>
+ * <h1 class="header">Functionality</h1>
+ * Distributed atomic long includes the following main functionality:
+ * <ul>
+ * <li>
+ * Method {@link #get()} synchronously gets current value of atomic long.
+ * </li>
+ * <li>
+ * Various {@code get..(..)} methods synchronously get current value of atomic 
long
+ * and increase or decrease value of atomic long.
+ * </li>
+ * <li>
+ * Method {@link #addAndGet(long l)} synchronously sums {@code l} with current 
value of atomic long
+ * and returns result.
+ * </li>
+ * <li>
+ * Method {@link #incrementAndGet()} synchronously increases value of atomic 
long and returns result.
+ * </li>
+ * <li>
+ * Method {@link #decrementAndGet()} synchronously decreases value of atomic 
long and returns result.
+ * </li>
+ * <li>
+ * Method {@link #getAndSet(long l)} synchronously gets current value of 
atomic long and sets {@code l}
+ * as value of atomic long.
+ * </li>
+ * </ul>
+ * All previously described methods have asynchronous analogs.
+ * <ul>
+ * <li>
+ * Method {@link #name()} gets name of atomic long.
+ * </li>
+ * </ul>
+ * <p>
+ * <h1 class="header">Creating Distributed Atomic Long</h1>
+ * Instance of distributed atomic long can be created by calling the following 
method:
+ * <ul>
+ *     <li>{@link CacheDataStructures#atomicLong(String, long, boolean)}</li>
+ * </ul>
+ * @see CacheDataStructures#atomicLong(String, long, boolean)
+ * @see CacheDataStructures#removeAtomicLong(String)
+ */
+public interface CacheAtomicLong {
+    /**
+     * Name of atomic long.
+     *
+     * @return Name of atomic long.
+     */
+    public String name();
+
+    /**
+     * Gets current value of atomic long.
+     *
+     * @return Current value of atomic long.
+     * @throws IgniteCheckedException If operation failed.
+     */
+    public long get() throws IgniteCheckedException;
+
+    /**
+     * Increments and gets current value of atomic long.
+     *
+     * @return Value.
+     * @throws IgniteCheckedException If operation failed.
+     */
+    public long incrementAndGet() throws IgniteCheckedException;
+
+    /**
+     * Gets and increments current value of atomic long.
+     *
+     * @return Value.
+     * @throws IgniteCheckedException If operation failed.
+     */
+    public long getAndIncrement() throws IgniteCheckedException;
+
+    /**
+     * Adds {@code l} and gets current value of atomic long.
+     *
+     * @param l Number which will be added.
+     * @return Value.
+     * @throws IgniteCheckedException If operation failed.
+     */
+    public long addAndGet(long l) throws IgniteCheckedException;
+
+    /**
+     * Gets current value of atomic long and adds {@code l}.
+     *
+     * @param l Number which will be added.
+     * @return Value.
+     * @throws IgniteCheckedException If operation failed.
+     */
+    public long getAndAdd(long l) throws IgniteCheckedException;
+
+    /**
+     * Decrements and gets current value of atomic long.
+     *
+     * @return Value.
+     * @throws IgniteCheckedException If operation failed.
+     */
+    public long decrementAndGet() throws IgniteCheckedException;
+
+    /**
+     * Gets and decrements current value of atomic long.
+     *
+     * @return Value.
+     * @throws IgniteCheckedException If operation failed.
+     */
+    public long getAndDecrement() throws IgniteCheckedException;
+
+    /**
+     * Gets current value of atomic long and sets new value {@code l} of 
atomic long.
+     *
+     * @param l New value of atomic long.
+     * @return Value.
+     * @throws IgniteCheckedException If operation failed.
+     */
+    public long getAndSet(long l) throws IgniteCheckedException;
+
+    /**
+     * Atomically compares current value to the expected value, and if they 
are equal, sets current value
+     * to new value.
+     *
+     * @param expVal Expected atomic long's value.
+     * @param newVal New atomic long's value to set if current value equal to 
expected value.
+     * @return {@code True} if comparison succeeded, {@code false} otherwise.
+     * @throws IgniteCheckedException If failed.
+     */
+    public boolean compareAndSet(long expVal, long newVal) throws 
IgniteCheckedException;
+
+    /**
+     * Gets status of atomic.
+     *
+     * @return {@code true} if atomic was removed from cache, {@code false} in 
other case.
+     */
+    public boolean removed();
+}

Reply via email to