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 2017/03/13 15:09:35 UTC

[1/8] ignite git commit: ignite-4705 Atomic cache protocol change: notify client node from backups

Repository: ignite
Updated Branches:
  refs/heads/ignite-2.0 f59f46d37 -> cbc472fe7


http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/IgniteCacheAtomicProtocolTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/IgniteCacheAtomicProtocolTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/IgniteCacheAtomicProtocolTest.java
new file mode 100644
index 0000000..6d90d0e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/IgniteCacheAtomicProtocolTest.java
@@ -0,0 +1,883 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheEntryProcessor;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.managers.communication.GridIoMessage;
+import org.apache.ignite.internal.processors.cache.GridCacheMessage;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessageV2;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheRebalanceMode.ASYNC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
+
+/**
+ *
+ */
+public class IgniteCacheAtomicProtocolTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final String TEST_CACHE = "testCache";
+
+    /** */
+    private boolean client;
+
+    /** */
+    private CacheConfiguration ccfg;
+
+    /** */
+    private boolean blockRebalance;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setConsistentId(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setMaxMissedClientHeartbeats(1000);
+
+        TestRecordingCommunicationSpi commSpi = new TestRecordingCommunicationSpi();
+
+        cfg.setCommunicationSpi(commSpi);
+
+        cfg.setClientMode(client);
+
+        if (ccfg != null)
+            cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /**
+     *
+     */
+    private void blockRebalance() {
+        for (Ignite node : G.allGrids()) {
+            testSpi(node).blockMessages(new IgnitePredicate<GridIoMessage>() {
+                @Override public boolean apply(GridIoMessage msg) {
+                    Object msg0 = msg.message();
+
+                    return (msg0 instanceof GridDhtPartitionSupplyMessage || msg0 instanceof GridDhtPartitionSupplyMessageV2)
+                        && ((GridCacheMessage)msg0).cacheId() == CU.cacheId(TEST_CACHE);
+                }
+            });
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        super.afterTest();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllPrimaryFailure1() throws Exception {
+        putAllPrimaryFailure(true, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllPrimaryFailure1_UnstableTopology() throws Exception {
+        blockRebalance = true;
+
+        putAllPrimaryFailure(true, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllPrimaryFailure2() throws Exception {
+        putAllPrimaryFailure(true, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllPrimaryFailure2_UnstableTopology() throws Exception {
+        blockRebalance = true;
+
+        putAllPrimaryFailure(true, true);
+    }
+
+    /**
+     * @param fail0 Fail node 0 flag.
+     * @param fail1 Fail node 1 flag.
+     * @throws Exception If failed.
+     */
+    private void putAllPrimaryFailure(boolean fail0, boolean fail1) throws Exception {
+        ccfg = cacheConfiguration(1, FULL_SYNC);
+
+        startServers(4);
+
+        client = true;
+
+        Ignite client = startGrid(4);
+
+        IgniteCache<Integer, Integer> nearCache = client.cache(TEST_CACHE);
+        IgniteCache<Integer, Integer> nearAsyncCache = nearCache.withAsync();
+
+        if (!blockRebalance)
+            awaitPartitionMapExchange();
+
+        Ignite srv0 = ignite(0);
+        Ignite srv1 = ignite(1);
+
+        Integer key1 = primaryKey(srv0.cache(TEST_CACHE));
+        Integer key2 = primaryKey(srv1.cache(TEST_CACHE));
+
+        Map<Integer, Integer> map = new HashMap<>();
+        map.put(key1, key1);
+        map.put(key2, key2);
+
+        assertEquals(2, map.size());
+
+        if (fail0) {
+            testSpi(client).blockMessages(GridNearAtomicFullUpdateRequest.class, srv0.name());
+            testSpi(client).blockMessages(GridNearAtomicCheckUpdateRequest.class, srv0.name());
+        }
+
+        if (fail1) {
+            testSpi(client).blockMessages(GridNearAtomicFullUpdateRequest.class, srv1.name());
+            testSpi(client).blockMessages(GridNearAtomicCheckUpdateRequest.class, srv1.name());
+        }
+
+        log.info("Start put [key1=" + key1 + ", key2=" + key2 + ']');
+
+        nearAsyncCache.putAll(map);
+
+        IgniteFuture<?> fut = nearAsyncCache.future();
+
+        U.sleep(500);
+
+        assertFalse(fut.isDone());
+
+        if (fail0)
+            stopGrid(0);
+
+        if (fail1)
+            stopGrid(1);
+
+        fut.get();
+
+        checkData(map);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllBackupFailure1() throws Exception {
+        putAllBackupFailure1();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllBackupFailure1_UnstableTopology() throws Exception {
+        blockRebalance = true;
+
+        putAllBackupFailure1();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void putAllBackupFailure1() throws Exception {
+        ccfg = cacheConfiguration(1, FULL_SYNC);
+
+        startServers(4);
+
+        client = true;
+
+        Ignite client = startGrid(4);
+
+        IgniteCache<Integer, Integer> nearCache = client.cache(TEST_CACHE);
+        IgniteCache<Integer, Integer> nearAsyncCache = nearCache.withAsync();
+
+        if (!blockRebalance)
+            awaitPartitionMapExchange();
+
+        Ignite srv0 = ignite(0);
+
+        List<Integer> keys = primaryKeys(srv0.cache(TEST_CACHE), 3);
+
+        Ignite backup = backup(client.affinity(TEST_CACHE), keys.get(0));
+
+        testSpi(backup).blockMessages(GridDhtAtomicNearResponse.class, client.name());
+
+        Map<Integer, Integer> map = new HashMap<>();
+
+        for (Integer key : keys)
+            map.put(key, key);
+
+        log.info("Start put [map=" + map + ']');
+
+        nearAsyncCache.putAll(map);
+
+        IgniteFuture<?> fut = nearAsyncCache.future();
+
+        U.sleep(500);
+
+        assertFalse(fut.isDone());
+
+        stopGrid(backup.name());
+
+        fut.get();
+
+        checkData(map);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutBackupFailure1() throws Exception {
+        putBackupFailure1();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutBackupFailure1_UnstableTopology() throws Exception {
+        blockRebalance = true;
+
+        putBackupFailure1();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void putBackupFailure1() throws Exception {
+        ccfg = cacheConfiguration(1, FULL_SYNC);
+
+        startServers(4);
+
+        client = true;
+
+        Ignite client = startGrid(4);
+
+        IgniteCache<Integer, Integer> nearCache = client.cache(TEST_CACHE);
+        IgniteCache<Integer, Integer> nearAsyncCache = nearCache.withAsync();
+
+        if (!blockRebalance)
+            awaitPartitionMapExchange();
+
+        Ignite srv0 = ignite(0);
+
+        Integer key = primaryKey(srv0.cache(TEST_CACHE));
+
+        Ignite backup = backup(client.affinity(TEST_CACHE), key);
+
+        testSpi(backup).blockMessages(GridDhtAtomicNearResponse.class, client.name());
+
+        log.info("Start put [key=" + key + ']');
+
+        nearAsyncCache.put(key, key);
+
+        IgniteFuture<?> fut = nearAsyncCache.future();
+
+        U.sleep(500);
+
+        assertFalse(fut.isDone());
+
+        stopGrid(backup.name());
+
+        fut.get();
+
+        checkData(F.asMap(key, key));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFullAsyncPutRemap() throws Exception {
+        fullAsyncRemap(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFullAsyncPutAllRemap() throws Exception {
+        fullAsyncRemap(true);
+    }
+
+    /**
+     * @param putAll Test putAll flag.
+     * @throws Exception If failed.
+     */
+    private void fullAsyncRemap(boolean putAll) throws Exception {
+        Ignite srv0 = startGrid(0);
+
+        client = true;
+
+        Ignite clientNode = startGrid(1);
+
+        client = false;
+
+        final IgniteCache<Integer, Integer> nearCache = clientNode.createCache(cacheConfiguration(1, FULL_ASYNC));
+
+        List<Integer> keys = primaryKeys(srv0.cache(TEST_CACHE), putAll ? 3 : 1);
+
+        testSpi(clientNode).blockMessages(GridNearAtomicSingleUpdateRequest.class, srv0.name());
+        testSpi(clientNode).blockMessages(GridNearAtomicFullUpdateRequest.class, srv0.name());
+
+        final Map<Integer, Integer> map = new HashMap<>();
+
+        for (Integer key : keys)
+            map.put(key, -key);
+
+        if (putAll)
+            nearCache.putAll(map);
+        else
+            nearCache.put(keys.get(0), map.get(keys.get(0)));
+
+        int nodeIdx = 2;
+
+        Affinity<Object> aff = clientNode.affinity(TEST_CACHE);
+
+        int keysMoved;
+
+        do {
+            startGrid(nodeIdx);
+
+            awaitPartitionMapExchange();
+
+            keysMoved = 0;
+
+            for (Integer key : keys) {
+                if (!aff.isPrimary(srv0.cluster().localNode(), key))
+                    keysMoved++;
+            }
+
+            if (keysMoved == keys.size())
+                break;
+
+            nodeIdx++;
+        }
+        while (nodeIdx < 10);
+
+        assertEquals(keys.size(), keysMoved);
+
+        testSpi(clientNode).stopBlock(true);
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                for (Integer key : map.keySet()) {
+                    if (nearCache.get(key) == null)
+                        return false;
+                }
+
+                return true;
+            }
+        }, 5000);
+
+        checkData(map);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutPrimarySync() throws Exception {
+        startGrids(2);
+
+        client = true;
+
+        Ignite clientNode = startGrid(2);
+
+        client = false;
+
+        final IgniteCache<Integer, Integer> nearCache = clientNode.createCache(cacheConfiguration(1, PRIMARY_SYNC));
+        IgniteCache<Integer, Integer> nearAsyncCache = nearCache.withAsync();
+
+        awaitPartitionMapExchange();
+
+        Ignite srv0 = grid(0);
+        final Ignite srv1 = grid(1);
+
+        final Integer key = primaryKey(srv0.cache(TEST_CACHE));
+
+        testSpi(srv0).blockMessages(GridDhtAtomicSingleUpdateRequest.class, srv1.name());
+
+        nearAsyncCache.put(key, key);
+
+        IgniteFuture<?> fut = nearAsyncCache.future();
+
+        fut.get(5, TimeUnit.SECONDS);
+
+        assertEquals(key, srv0.cache(TEST_CACHE).get(key));
+
+        assertNull(srv1.cache(TEST_CACHE).localPeek(key));
+
+        testSpi(srv0).stopBlock(true);
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return srv1.cache(TEST_CACHE).localPeek(key) != null;
+            }
+        }, 5000);
+
+        checkData(F.asMap(key, key));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutNearNodeFailure() throws Exception {
+        startGrids(2);
+
+        client = true;
+
+        Ignite clientNode = startGrid(2);
+
+        final IgniteCache<Integer, Integer> nearCache = clientNode.createCache(cacheConfiguration(1, FULL_SYNC));
+        IgniteCache<Integer, Integer> nearAsyncCache = nearCache.withAsync();
+
+        awaitPartitionMapExchange();
+
+        final Ignite srv0 = grid(0);
+        final Ignite srv1 = grid(1);
+
+        final Integer key = primaryKey(srv0.cache(TEST_CACHE));
+
+        nearAsyncCache.put(key, key);
+
+        testSpi(srv1).blockMessages(GridDhtAtomicNearResponse.class, clientNode.name());
+
+        stopGrid(2);
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return ((IgniteKernal)srv0).context().cache().context().mvcc().atomicFuturesCount() == 0;
+            }
+        }, 5000);
+
+        assertEquals(0, ((IgniteKernal)srv0).context().cache().context().mvcc().atomicFuturesCount());
+        assertEquals(0, ((IgniteKernal)srv1).context().cache().context().mvcc().atomicFuturesCount());
+
+        checkData(F.asMap(key, key));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllNearNodeFailure() throws Exception {
+        final int SRVS = 4;
+
+        startGrids(SRVS);
+
+        client = true;
+
+        Ignite clientNode = startGrid(SRVS);
+
+        final IgniteCache<Integer, Integer> nearCache = clientNode.createCache(cacheConfiguration(1, FULL_SYNC));
+        IgniteCache<Integer, Integer> nearAsyncCache = nearCache.withAsync();
+
+        awaitPartitionMapExchange();
+
+        for (int i = 0; i < SRVS; i++)
+            testSpi(grid(i)).blockMessages(GridDhtAtomicNearResponse.class, clientNode.name());
+
+        final Map<Integer, Integer> map = new HashMap<>();
+
+        for (int i = 0; i < 100; i++)
+            map.put(i, i);
+
+        nearAsyncCache.putAll(map);
+
+        boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                IgniteCache cache = ignite(0).cache(TEST_CACHE);
+
+                for (Integer key : map.keySet()) {
+                    if (cache.get(key) == null)
+                        return false;
+                }
+
+                return true;
+            }
+        }, 5000);
+
+        assertTrue(wait);
+
+        stopGrid(SRVS);
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                for (int i = 0; i < SRVS; i++) {
+                    if (grid(i).context().cache().context().mvcc().atomicFuturesCount() != 0)
+                        return false;
+                }
+
+                return true;
+            }
+        }, 5000);
+
+        for (int i = 0; i < SRVS; i++)
+            assertEquals(0, grid(i).context().cache().context().mvcc().atomicFuturesCount());
+
+        checkData(map);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCacheOperations0() throws Exception {
+        cacheOperations(0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCacheOperations_UnstableTopology0() throws Exception {
+        blockRebalance = true;
+
+        cacheOperations(0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCacheOperations1() throws Exception {
+        cacheOperations(1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCacheOperations_UnstableTopology1() throws Exception {
+        blockRebalance = true;
+
+        cacheOperations(1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCacheOperations2() throws Exception {
+        cacheOperations(2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCacheOperations_UnstableTopology2() throws Exception {
+        blockRebalance = true;
+
+        cacheOperations(2);
+    }
+
+    /**
+     * @param backups Number of backups.
+     * @throws Exception If failed.
+     */
+    private void cacheOperations(int backups) throws Exception {
+        ccfg = cacheConfiguration(backups, FULL_SYNC);
+
+        final int SRVS = 4;
+
+        startServers(SRVS);
+
+        client = true;
+
+        Ignite clientNode = startGrid(SRVS);
+
+        final IgniteCache<Integer, Integer> nearCache = clientNode.cache(TEST_CACHE);
+
+        Integer key = primaryKey(ignite(0).cache(TEST_CACHE));
+
+        nearCache.replace(key, 1);
+
+        nearCache.remove(key);
+
+        nearCache.invoke(key, new SetValueEntryProcessor(null));
+
+        Map<Integer, SetValueEntryProcessor> map = new HashMap<>();
+
+        List<Integer> keys = primaryKeys(ignite(0).cache(TEST_CACHE), 2);
+
+        map.put(keys.get(0), new SetValueEntryProcessor(1));
+        map.put(keys.get(1), new SetValueEntryProcessor(null));
+
+        nearCache.invokeAll(map);
+
+        Set<Integer> rmvAllKeys = new HashSet<>();
+
+        for (int i = 0; i < 100; i++) {
+            nearCache.put(i, i);
+
+            if (i % 2 == 0)
+                rmvAllKeys.add(i);
+        }
+
+        nearCache.removeAll(rmvAllKeys);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutMissedDhtRequest_UnstableTopology() throws Exception {
+        blockRebalance = true;
+
+        ccfg = cacheConfiguration(1, FULL_SYNC);
+
+        startServers(4);
+
+        client = true;
+
+        Ignite client = startGrid(4);
+
+        IgniteCache<Integer, Integer> nearCache = client.cache(TEST_CACHE);
+        IgniteCache<Integer, Integer> nearAsyncCache = nearCache.withAsync();
+
+        testSpi(ignite(0)).blockMessages(new IgnitePredicate<GridIoMessage>() {
+            @Override public boolean apply(GridIoMessage msg) {
+                return msg.message() instanceof GridDhtAtomicAbstractUpdateRequest;
+            }
+        });
+
+        Integer key = primaryKey(ignite(0).cache(TEST_CACHE));
+
+        log.info("Start put [key=" + key + ']');
+
+        nearAsyncCache.put(key, key);
+        IgniteFuture<?> fut = nearAsyncCache.future();
+
+        U.sleep(500);
+
+        assertFalse(fut.isDone());
+
+        stopGrid(0);
+
+        fut.get();
+
+        checkData(F.asMap(key, key));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllMissedDhtRequest_UnstableTopology1() throws Exception {
+        putAllMissedDhtRequest_UnstableTopology(true, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllMissedDhtRequest_UnstableTopology2() throws Exception {
+        putAllMissedDhtRequest_UnstableTopology(true, true);
+    }
+
+    /**
+     * @param fail0 Fail node 0 flag.
+     * @param fail1 Fail node 1 flag.
+     * @throws Exception If failed.
+     */
+    private void putAllMissedDhtRequest_UnstableTopology(boolean fail0, boolean fail1) throws Exception {
+        blockRebalance = true;
+
+        ccfg = cacheConfiguration(1, FULL_SYNC);
+
+        startServers(4);
+
+        client = true;
+
+        Ignite client = startGrid(4);
+
+        IgniteCache<Integer, Integer> nearCache = client.cache(TEST_CACHE);
+        IgniteCache<Integer, Integer> nearAsyncCache = nearCache.withAsync();
+
+        if (fail0) {
+            testSpi(ignite(0)).blockMessages(new IgnitePredicate<GridIoMessage>() {
+                @Override public boolean apply(GridIoMessage msg) {
+                    return msg.message() instanceof GridDhtAtomicAbstractUpdateRequest;
+                }
+            });
+        }
+        if (fail1) {
+            testSpi(ignite(2)).blockMessages(new IgnitePredicate<GridIoMessage>() {
+                @Override public boolean apply(GridIoMessage msg) {
+                    return msg.message() instanceof GridDhtAtomicAbstractUpdateRequest;
+                }
+            });
+        }
+
+        Integer key1 = primaryKey(ignite(0).cache(TEST_CACHE));
+        Integer key2 = primaryKey(ignite(2).cache(TEST_CACHE));
+
+        log.info("Start put [key1=" + key1 + ", key2=" + key1 + ']');
+
+        Map<Integer, Integer> map = new HashMap<>();
+        map.put(key1, 10);
+        map.put(key2, 20);
+
+        nearAsyncCache.putAll(map);
+        IgniteFuture<?> fut = nearAsyncCache.future();
+
+        U.sleep(500);
+
+        assertFalse(fut.isDone());
+
+        if (fail0)
+            stopGrid(0);
+        if (fail1)
+            stopGrid(2);
+
+        fut.get();
+
+        checkData(map);
+    }
+
+    /**
+     * @param expData Expected cache data.
+     */
+    private void checkData(Map<Integer, Integer> expData) {
+        assert !expData.isEmpty();
+
+        List<Ignite> nodes = G.allGrids();
+
+        assertFalse(nodes.isEmpty());
+
+        for (Ignite node : nodes) {
+            IgniteCache<Integer, Integer> cache = node.cache(TEST_CACHE);
+
+            for (Map.Entry<Integer, Integer> e : expData.entrySet()) {
+                assertEquals("Invalid value [key=" + e.getKey() + ", node=" + node.name() + ']',
+                    e.getValue(),
+                    cache.get(e.getKey()));
+            }
+        }
+    }
+
+    /**
+     * @param aff Affinity.
+     * @param key Key.
+     * @return Backup node for given key.
+     */
+    private Ignite backup(Affinity<Object> aff, Object key) {
+        for (Ignite ignite : G.allGrids()) {
+            ClusterNode node = ignite.cluster().localNode();
+
+            if (aff.isPrimaryOrBackup(node, key) && !aff.isPrimary(node, key))
+                return ignite;
+        }
+
+        fail("Failed to find backup for key: " + key);
+
+        return null;
+    }
+
+    /**
+     * @param node Node.
+     * @return Node communication SPI.
+     */
+    private TestRecordingCommunicationSpi testSpi(Ignite node) {
+        return (TestRecordingCommunicationSpi)node.configuration().getCommunicationSpi();
+    }
+
+    /**
+     * @param backups Number of backups.
+     * @param writeSync Cache write synchronization mode.
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<Integer, Integer> cacheConfiguration(int backups,
+        CacheWriteSynchronizationMode writeSync) {
+        CacheConfiguration<Integer, Integer> ccfg = new CacheConfiguration<>();
+
+        ccfg.setName(TEST_CACHE);
+        ccfg.setAtomicityMode(ATOMIC);
+        ccfg.setWriteSynchronizationMode(writeSync);
+        ccfg.setBackups(backups);
+        ccfg.setRebalanceMode(ASYNC);
+
+        return ccfg;
+    }
+
+    /**
+     * @param cnt Number of server nodes.
+     * @throws Exception If failed.
+     */
+    private void startServers(int cnt) throws Exception {
+        startGrids(cnt - 1);
+
+        awaitPartitionMapExchange();
+
+        if (blockRebalance)
+            blockRebalance();
+
+        startGrid(cnt - 1);
+    }
+
+    /**
+     *
+     */
+    public static class SetValueEntryProcessor implements CacheEntryProcessor<Integer, Integer, Object> {
+        /** */
+        private Integer val;
+
+        /**
+         * @param val Value.
+         */
+        SetValueEntryProcessor(Integer val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object process(MutableEntry<Integer, Integer> entry, Object... args) {
+            if (val != null)
+                entry.setValue(val);
+
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest.java
deleted file mode 100644
index 9057507..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.distributed.near;
-
-import org.apache.ignite.configuration.IgniteConfiguration;
-
-/**
- *
- */
-public class GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest extends
-    GridCacheAtomicPrimaryWriteOrderMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        cfg.setStripedPoolSize(-1);
-
-        return cfg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
index 7646741..9505b24 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
@@ -40,11 +40,13 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheModuloAffinityFunction;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
@@ -147,7 +149,7 @@ public class GridCacheNearReadersSelfTest extends GridCommonAbstractTest {
         startGrids();
 
         ClusterNode n1 = F.first(aff.nodes(aff.partition(1), grid(0).cluster().nodes()));
-        ClusterNode n2 = F.first(aff.nodes(aff.partition(2), grid(0).cluster().nodes()));
+        final ClusterNode n2 = F.first(aff.nodes(aff.partition(2), grid(0).cluster().nodes()));
 
         assertNotNull(n1);
         assertNotNull(n2);
@@ -164,7 +166,7 @@ public class GridCacheNearReadersSelfTest extends GridCommonAbstractTest {
         assertNull(cache1.getAndPut(1, "v1"));
         assertNull(cache1.getAndPut(2, "v2"));
 
-        GridDhtCacheEntry e1 = (GridDhtCacheEntry)dht(cache1).entryEx(1);
+        final GridDhtCacheEntry e1 = (GridDhtCacheEntry)dht(cache1).entryEx(1);
         GridDhtCacheEntry e2 = (GridDhtCacheEntry)dht(cache2).entryEx(2);
 
         assertNotNull(e1.readers());
@@ -207,6 +209,17 @@ public class GridCacheNearReadersSelfTest extends GridCommonAbstractTest {
 
         assertNotNull(cache1.getAndPut(1, "z1"));
 
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                try {
+                    return !e1.readers().contains(n2.id());
+                }
+                catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+            }
+        }, 5000);
+
         // Node 1 still has node2 in readers map.
         assertFalse(e1.readers().contains(n2.id()));
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest.java
deleted file mode 100644
index 7ba3144..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.distributed.near;
-
-import org.apache.ignite.configuration.IgniteConfiguration;
-
-/**
- *
- */
-public class GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest extends
-    GridCachePartitionedMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        cfg.setStripedPoolSize(-1);
-
-        return cfg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFilterListenerTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFilterListenerTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFilterListenerTest.java
index 958bb5c..39e995a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFilterListenerTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFilterListenerTest.java
@@ -771,7 +771,7 @@ public class CacheContinuousQueryAsyncFilterListenerTest extends GridCommonAbstr
 
     /** {@inheritDoc} */
     @Override protected long getTestTimeout() {
-        return TimeUnit.SECONDS.toMillis(15);
+        return TimeUnit.SECONDS.toMillis(2 * 60);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
index cb8e755..716bb0d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
@@ -258,15 +258,15 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
                 qryClnCache.put(key, -1);
 
             qryClnCache.put(keys.get(0), 100);
-        }
 
-        GridTestUtils.waitForCondition(new GridAbsPredicate() {
-            @Override public boolean apply() {
-                return lsnr.evts.size() == 1;
-            }
-        }, 5000);
+            GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                @Override public boolean apply() {
+                    return lsnr.evts.size() == 1;
+                }
+            }, 5000);
 
-        assertEquals(lsnr.evts.size(), 1);
+            assertEquals(lsnr.evts.size(), 1);
+        }
     }
 
     /**
@@ -480,7 +480,7 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
                 if (rnd.nextBoolean())
                     cache = qryClient.cache(null);
                 else {
-                    for (int j = 0; j < 10; j++) {
+                    for (int j = 0; j < 1000; j++) {
                         int nodeIdx = rnd.nextInt(SRV_NODES);
 
                         if (killedNode != nodeIdx) {
@@ -1150,11 +1150,10 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
         boolean lostAllow, boolean wait) throws Exception {
         if (wait) {
             GridTestUtils.waitForCondition(new PA() {
-                @Override
-                public boolean apply() {
+                @Override public boolean apply() {
                     return expEvts.size() == lsnr.size();
                 }
-            }, 2000L);
+            }, 10_000L);
         }
 
         synchronized (lsnr) {
@@ -1970,9 +1969,9 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
 
                                     GridTestUtils.waitForCondition(new PA() {
                                         @Override public boolean apply() {
-                                            return lsnr.size() <= size;
+                                            return lsnr.size() >= size;
                                         }
-                                    }, 2000L);
+                                    }, 10_000L);
 
                                     List<T3<Object, Object, Object>> expEvts0 = new ArrayList<>();
 
@@ -2300,16 +2299,16 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
 
                 log.info("Batch loaded. Iteration: " + iteration);
 
-                final long cnt = lsnr.count();
-
                 final long expCnt = putCnt * stableNodeCnt + ignoredDupEvts;
 
                 GridTestUtils.waitForCondition(new GridAbsPredicate() {
                     @Override public boolean apply() {
-                        return cnt == expCnt;
+                        return lsnr.count() == expCnt;
                     }
                 }, 6_000);
 
+                final long cnt = lsnr.count();
+
                 if (cnt != expCnt) {
                     StringBuilder sb = new StringBuilder();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java
index ecfb4e8..efac24a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java
@@ -132,7 +132,7 @@ public class IgniteCacheContinuousQueryImmutableEntryTest extends GridCommonAbst
         CacheContinuousQueryEntry e0 = new CacheContinuousQueryEntry(
             1,
             EventType.UPDATED,
-            new KeyCacheObjectImpl(1, new byte[] {0, 0, 0, 1}),
+            new KeyCacheObjectImpl(1, new byte[] {0, 0, 0, 1}, 1),
             new CacheObjectImpl(2, new byte[] {0, 0, 0, 2}),
             new CacheObjectImpl(2, new byte[] {0, 0, 0, 3}),
             true,

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/internal/util/future/nio/GridNioEmbeddedFutureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/future/nio/GridNioEmbeddedFutureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/future/nio/GridNioEmbeddedFutureSelfTest.java
index 5591f2c..b7b6966 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/future/nio/GridNioEmbeddedFutureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/future/nio/GridNioEmbeddedFutureSelfTest.java
@@ -32,7 +32,7 @@ public class GridNioEmbeddedFutureSelfTest extends GridCommonAbstractTest {
      */
     public void testNioEmbeddedFuture() throws Exception {
         // Original future.
-        final GridNioFutureImpl<Integer> origFut = new GridNioFutureImpl<>();
+        final GridNioFutureImpl<Integer> origFut = new GridNioFutureImpl<>(null);
 
         // Embedded future to test.
         final GridNioEmbeddedFuture<Integer> embFut = new GridNioEmbeddedFuture<>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/internal/util/future/nio/GridNioFutureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/future/nio/GridNioFutureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/future/nio/GridNioFutureSelfTest.java
index 684ae01..44a1eff 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/future/nio/GridNioFutureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/future/nio/GridNioFutureSelfTest.java
@@ -35,24 +35,23 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
  * Test for NIO future.
  */
 public class GridNioFutureSelfTest extends GridCommonAbstractTest {
-
     /**
      * @throws Exception If failed.
      */
     public void testOnDone() throws Exception {
-        GridNioFutureImpl<String> fut = new GridNioFutureImpl<>();
+        GridNioFutureImpl<String> fut = new GridNioFutureImpl<>(null);
 
         fut.onDone();
 
         assertNull(fut.get());
 
-        fut = new GridNioFutureImpl<>();
+        fut = new GridNioFutureImpl<>(null);
 
         fut.onDone("test");
 
         assertEquals("test", fut.get());
 
-        fut = new GridNioFutureImpl<>();
+        fut = new GridNioFutureImpl<>(null);
 
         fut.onDone(new IgniteCheckedException("TestMessage"));
 
@@ -64,7 +63,7 @@ public class GridNioFutureSelfTest extends GridCommonAbstractTest {
             }
         }, IgniteCheckedException.class, "TestMessage");
 
-        fut = new GridNioFutureImpl<>();
+        fut = new GridNioFutureImpl<>(null);
 
         fut.onDone("test", new IgniteCheckedException("TestMessage"));
 
@@ -76,7 +75,7 @@ public class GridNioFutureSelfTest extends GridCommonAbstractTest {
             }
         }, IgniteCheckedException.class, "TestMessage");
 
-        fut = new GridNioFutureImpl<>();
+        fut = new GridNioFutureImpl<>(null);
 
         fut.onDone("test");
 
@@ -86,12 +85,12 @@ public class GridNioFutureSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @throws Exception
+     * @throws Exception If failed.
      */
     public void testOnCancelled() throws Exception {
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
-                GridNioFutureImpl<String> fut = new GridNioFutureImpl<>();
+                GridNioFutureImpl<String> fut = new GridNioFutureImpl<>(null);
 
                 fut.onCancelled();
 
@@ -101,7 +100,7 @@ public class GridNioFutureSelfTest extends GridCommonAbstractTest {
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
-                GridNioFutureImpl<String> fut = new GridNioFutureImpl<>();
+                GridNioFutureImpl<String> fut = new GridNioFutureImpl<>(null);
 
                 fut.onCancelled();
 
@@ -116,7 +115,7 @@ public class GridNioFutureSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testListenSyncNotify() throws Exception {
-        GridNioFutureImpl<String> fut = new GridNioFutureImpl<>();
+        GridNioFutureImpl<String> fut = new GridNioFutureImpl<>(null);
 
         int lsnrCnt = 10;
 
@@ -167,9 +166,9 @@ public class GridNioFutureSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testGet() throws Exception {
-        GridNioFutureImpl<Object> unfinished = new GridNioFutureImpl<>();
-        GridNioFutureImpl<Object> finished = new GridNioFutureImpl<>();
-        GridNioFutureImpl<Object> cancelled = new GridNioFutureImpl<>();
+        GridNioFutureImpl<Object> unfinished = new GridNioFutureImpl<>(null);
+        GridNioFutureImpl<Object> finished = new GridNioFutureImpl<>(null);
+        GridNioFutureImpl<Object> cancelled = new GridNioFutureImpl<>(null);
 
         finished.onDone("Finished");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/internal/util/nio/impl/GridNioFilterChainSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/impl/GridNioFilterChainSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/impl/GridNioFilterChainSelfTest.java
index d403784..e6aab9f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/impl/GridNioFilterChainSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/impl/GridNioFilterChainSelfTest.java
@@ -21,6 +21,7 @@ import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter;
 import org.apache.ignite.internal.util.nio.GridNioFilterAdapter;
 import org.apache.ignite.internal.util.nio.GridNioFilterChain;
@@ -30,6 +31,7 @@ import org.apache.ignite.internal.util.nio.GridNioRecoveryDescriptor;
 import org.apache.ignite.internal.util.nio.GridNioServerListener;
 import org.apache.ignite.internal.util.nio.GridNioServerListenerAdapter;
 import org.apache.ignite.internal.util.nio.GridNioSession;
+import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
@@ -114,7 +116,7 @@ public class GridNioFilterChainSelfTest extends GridCommonAbstractTest {
                 proceedExceptionCaught(ses, ex);
             }
 
-            @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) {
+            @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut, IgniteInClosure<IgniteException> ackC) {
                 sndEvt.compareAndSet(null, ses.<String>meta(MESSAGE_WRITE_META_NAME));
 
                 sndMsgObj.compareAndSet(null, msg);
@@ -155,7 +157,7 @@ public class GridNioFilterChainSelfTest extends GridCommonAbstractTest {
         chain.onSessionIdleTimeout(ses);
         chain.onSessionWriteTimeout(ses);
         assertNull(chain.onSessionClose(ses));
-        assertNull(chain.onSessionWrite(ses, snd, true));
+        assertNull(chain.onSessionWrite(ses, snd, true, null));
 
         assertEquals("DCBA", connectedEvt.get());
         assertEquals("DCBA", disconnectedEvt.get());
@@ -210,10 +212,10 @@ public class GridNioFilterChainSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) throws IgniteCheckedException {
+        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut, IgniteInClosure<IgniteException> ackC) throws IgniteCheckedException {
             chainMeta(ses, MESSAGE_WRITE_META_NAME);
 
-            return proceedSessionWrite(ses, msg, fut);
+            return proceedSessionWrite(ses, msg, fut, ackC);
         }
 
         /** {@inheritDoc} */
@@ -349,7 +351,7 @@ public class GridNioFilterChainSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void sendNoFuture(Object msg) {
+        @Override public void sendNoFuture(Object msg, IgniteInClosure<IgniteException> ackC) {
             // No-op.
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java
index ab21165..a59b6d1 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java
@@ -118,7 +118,7 @@ public class GridFileSwapSpaceSpiSelfTest extends GridSwapSpaceSpiAbstractSelfTe
      * @return Swap key.
      */
     private SwapKey key(int i) {
-        return new SwapKey(new KeyCacheObjectImpl(i, U.intToBytes(i)), i % 11, U.intToBytes(i));
+        return new SwapKey(new KeyCacheObjectImpl(i, U.intToBytes(i), i), i % 11, U.intToBytes(i));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
index 17757ab..96a8a33 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
@@ -37,6 +37,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePart
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledOffHeapTieredFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledOffHeapTieredMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.CachePartitionedMultiNodeLongTxTimeoutFullApiTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicClientOnlyFairAffinityMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicClientOnlyMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicClientOnlyMultiNodeP2PDisabledFullApiSelfTest;
@@ -57,7 +58,6 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAto
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderFairAffinityMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderMultiNodeP2PDisabledFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderOffHeapFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderOffHeapTieredFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWrityOrderOffHeapMultiNodeFullApiSelfTest;
@@ -76,9 +76,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePar
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeCounterSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeP2PDisabledFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedNearOnlyNoPrimaryFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapTieredFullApiSelfTest;
@@ -230,10 +228,6 @@ public class IgniteCacheFullApiSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheReplicatedFullApiMultithreadedSelfTest.class);
         suite.addTestSuite(GridCachePartitionedFullApiMultithreadedSelfTest.class);
 
-        // Disabled striped pool.
-        suite.addTestSuite(GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest.class);
-        suite.addTestSuite(GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest.class);
-
         // Other.
         suite.addTestSuite(GridCacheClearSelfTest.class);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
index 03204e2..6fc6846 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
@@ -32,6 +32,7 @@ import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinity
 import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentNodeJoinValidationTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheTxIteratorSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.IgniteCacheAtomicProtocolTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheSyncRebalanceModeSelfTest;
 import org.apache.ignite.internal.processors.cache.store.IgniteCacheWriteBehindNoUpdateSelfTest;
 
@@ -66,6 +67,8 @@ public class IgniteCacheTestSuite5 extends TestSuite {
 
         suite.addTestSuite(CacheRebalancingSelfTest.class);
 
+        suite.addTestSuite(IgniteCacheAtomicProtocolTest.class);
+
         suite.addTestSuite(PartitionsExchangeOnDiscoveryHistoryOverflowTest.class);
 
         return suite;

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
index 9204c97..8a20eec 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
@@ -34,6 +34,7 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.hadoop.message.HadoopMessage;
@@ -63,6 +64,7 @@ import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.Nullable;
@@ -1306,11 +1308,14 @@ public class HadoopExternalCommunication {
         }
 
         /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) throws IgniteCheckedException {
+        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses,
+            Object msg,
+            boolean fut,
+            IgniteInClosure<IgniteException> ackC) throws IgniteCheckedException {
             if (ses.meta(PROCESS_META) == null && !(msg instanceof ProcessHandshakeMessage))
                 log.warning("Writing message before handshake has finished [ses=" + ses + ", msg=" + msg + ']');
 
-            return proceedSessionWrite(ses, msg, fut);
+            return proceedSessionWrite(ses, msg, fut, ackC);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java
index 3f33fb7..08981af 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java
@@ -24,6 +24,7 @@ import java.nio.ByteOrder;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.util.ipc.IpcEndpoint;
 import org.apache.ignite.internal.util.nio.GridNioFilter;
@@ -34,6 +35,7 @@ import org.apache.ignite.internal.util.nio.GridNioFuture;
 import org.apache.ignite.internal.util.nio.GridNioServerListener;
 import org.apache.ignite.internal.util.nio.GridNioSession;
 import org.apache.ignite.internal.util.nio.GridNioSessionImpl;
+import org.apache.ignite.lang.IgniteInClosure;
 
 /**
  * Allows to re-use existing {@link GridNioFilter}s on IPC (specifically shared memory IPC)
@@ -190,7 +192,10 @@ public class HadoopIpcToNioAdapter<T> {
         }
 
         /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) {
+        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses,
+            Object msg,
+            boolean fut,
+            IgniteInClosure<IgniteException> ackC) {
             assert ses == HadoopIpcToNioAdapter.this.ses : "ses=" + ses +
                 ", this.ses=" + HadoopIpcToNioAdapter.this.ses;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java
index 24bba88..d90a900 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java
@@ -18,11 +18,13 @@
 package org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication;
 
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.processors.hadoop.message.HadoopMessage;
 import org.apache.ignite.internal.util.nio.GridNioFilterAdapter;
 import org.apache.ignite.internal.util.nio.GridNioFuture;
 import org.apache.ignite.internal.util.nio.GridNioSession;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.marshaller.Marshaller;
 
 /**
@@ -57,12 +59,16 @@ public class HadoopMarshallerFilter extends GridNioFilterAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) throws IgniteCheckedException {
+    @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses,
+        Object msg,
+        boolean fut,
+        IgniteInClosure<IgniteException> ackC) throws IgniteCheckedException {
         assert msg instanceof HadoopMessage : "Invalid message type: " + msg;
 
-        return proceedSessionWrite(ses, U.marshal(marsh, msg), fut);
+        return proceedSessionWrite(ses, U.marshal(marsh, msg), fut, ackC);
     }
 
+    /** {@inheritDoc} */
     @Override public void onMessageReceived(GridNioSession ses, Object msg) throws IgniteCheckedException {
         assert msg instanceof byte[];
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutBenchmark.java
index 40e563c..8d15e5e 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutBenchmark.java
@@ -22,7 +22,7 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.yardstick.cache.model.SampleValue;
 
 /**
- * Ignite benchmark that performs invoke operations.
+ * Ignite benchmark that performs getAndPut operations.
  */
 public class IgniteGetAndPutBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutTxBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutTxBenchmark.java
index 49ae985..0a3794c 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutTxBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutTxBenchmark.java
@@ -27,7 +27,7 @@ import org.apache.ignite.yardstick.cache.model.SampleValue;
 import org.yardstickframework.BenchmarkConfiguration;
 
 /**
- * Ignite benchmark that performs invoke operations.
+ * Ignite benchmark that performs getAndPut operations.
  */
 public class IgniteGetAndPutTxBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
     /** */


[4/8] ignite git commit: ignite-4705 Atomic cache protocol change: notify client node from backups

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
index 0a816a7..930c4af 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
@@ -17,9 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 
-import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import javax.cache.expiry.ExpiryPolicy;
@@ -41,36 +40,29 @@ import org.apache.ignite.internal.processors.cache.GridCacheTryPutFailedExceptio
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearAtomicCache;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
-import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.lang.IgniteProductVersion;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM;
 
 /**
  * DHT atomic cache near update future.
  */
 public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpdateFuture {
-    /** */
-    private static final IgniteProductVersion SINGLE_UPDATE_REQUEST = IgniteProductVersion.fromString("1.7.4");
-
     /** Keys */
     private Object key;
 
     /** Values. */
-    @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized"})
     private Object val;
 
-    /** Not null is operation is mapped to single node. */
-    private GridNearAtomicAbstractUpdateRequest req;
+    /** */
+    private PrimaryRequestState reqState;
 
     /**
      * @param cctx Cache context.
@@ -110,8 +102,21 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
         int remapCnt,
         boolean waitTopFut
     ) {
-        super(cctx, cache, syncMode, op, invokeArgs, retval, rawRetval, expiryPlc, filter, subjId, taskNameHash,
-            skipStore, keepBinary, remapCnt, waitTopFut);
+        super(cctx,
+            cache,
+            syncMode,
+            op,
+            invokeArgs,
+            retval,
+            rawRetval,
+            expiryPlc,
+            filter,
+            subjId,
+            taskNameHash,
+            skipStore,
+            keepBinary,
+            remapCnt,
+            waitTopFut);
 
         assert subjId != null;
 
@@ -120,52 +125,63 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
     }
 
     /** {@inheritDoc} */
-    @Override public GridCacheVersion version() {
+    @Override public Long id() {
         synchronized (mux) {
-            return futVer;
+            return futId;
         }
     }
 
     /** {@inheritDoc} */
     @Override public boolean onNodeLeft(UUID nodeId) {
-        GridNearAtomicUpdateResponse res = null;
+        GridCacheReturn opRes0 = null;
+        CachePartialUpdateCheckedException err0 = null;
+        AffinityTopologyVersion remapTopVer0 = null;
 
-        GridNearAtomicAbstractUpdateRequest req;
+        GridNearAtomicCheckUpdateRequest checkReq = null;
+
+        boolean rcvAll = false;
 
         synchronized (mux) {
-            req = this.req != null && this.req.nodeId().equals(nodeId) ? this.req : null;
+            if (reqState == null)
+                return false;
 
-            if (req != null && req.response() == null) {
-                res = new GridNearAtomicUpdateResponse(cctx.cacheId(),
-                    nodeId,
-                    req.futureVersion(),
-                    cctx.deploymentEnabled());
+            if (reqState.req.nodeId.equals(nodeId)) {
+                GridNearAtomicAbstractUpdateRequest req = reqState.onPrimaryFail();
+
+                if (req != null) {
+                    GridNearAtomicUpdateResponse res = primaryFailedResponse(req);
 
-                ClusterTopologyCheckedException e = new ClusterTopologyCheckedException("Primary node left grid " +
-                    "before response is received: " + nodeId);
+                    rcvAll = true;
 
-                e.retryReadyFuture(cctx.shared().nextAffinityReadyFuture(req.topologyVersion()));
+                    reqState.onPrimaryResponse(res, cctx);
 
-                res.addFailedKeys(req.keys(), e);
+                    onPrimaryError(req, res);
+                }
             }
-        }
+            else {
+                DhtLeftResult res = reqState.onDhtNodeLeft(nodeId);
 
-        if (res != null) {
-            if (msgLog.isDebugEnabled()) {
-                msgLog.debug("Near update single fut, node left [futId=" + req.futureVersion() +
-                    ", writeVer=" + req.updateVersion() +
-                    ", node=" + nodeId + ']');
+                if (res == DhtLeftResult.DONE)
+                    rcvAll = true;
+                else if (res == DhtLeftResult.ALL_RCVD_CHECK_PRIMARY)
+                    checkReq = new GridNearAtomicCheckUpdateRequest(reqState.req);
+                else
+                    return false;
             }
 
-            onResult(nodeId, res, true);
+            if (rcvAll) {
+                opRes0 = opRes;
+                err0 = err;
+                remapTopVer0 = onAllReceived();
+            }
         }
 
-        return false;
-    }
+        if (checkReq != null)
+            sendCheckUpdateRequest(checkReq);
+        else if (rcvAll)
+            finishUpdateFuture(opRes0, err0, remapTopVer0);
 
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Void> completeFuture(AffinityTopologyVersion topVer) {
-        return null;
+        return false;
     }
 
     /** {@inheritDoc} */
@@ -175,15 +191,14 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
 
         GridCacheReturn ret = (GridCacheReturn)res;
 
-        Object retval =
-            res == null ? null : rawRetval ? ret : (this.retval || op == TRANSFORM) ?
-                cctx.unwrapBinaryIfNeeded(ret.value(), keepBinary) : ret.success();
+        Object retval = res == null ? null : rawRetval ? ret : (this.retval || op == TRANSFORM) ?
+            cctx.unwrapBinaryIfNeeded(ret.value(), keepBinary) : ret.success();
 
         if (op == TRANSFORM && retval == null)
             retval = Collections.emptyMap();
 
         if (super.onDone(retval, err)) {
-            GridCacheVersion futVer = onFutureDone();
+            Long futVer = onFutureDone();
 
             if (futVer != null)
                 cctx.mvcc().removeAtomicFuture(futVer);
@@ -195,112 +210,103 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
     }
 
     /** {@inheritDoc} */
+    @Override public void onDhtResponse(UUID nodeId, GridDhtAtomicNearResponse res) {
+        GridCacheReturn opRes0;
+        CachePartialUpdateCheckedException err0;
+        AffinityTopologyVersion remapTopVer0;
+
+        synchronized (mux) {
+            if (futId == null || futId != res.futureId())
+                return;
+
+            assert reqState != null;
+            assert reqState.req.nodeId().equals(res.primaryId());
+
+            if (opRes == null && res.hasResult())
+                opRes = res.result();
+
+            if (reqState.onDhtResponse(nodeId, res)) {
+                opRes0 = opRes;
+                err0 = err;
+                remapTopVer0 = onAllReceived();
+            }
+            else
+                return;
+        }
+
+        UpdateErrors errors = res.errors();
+
+        if (errors != null) {
+            assert errors.error() != null;
+
+            onDone(errors.error());
+
+            return;
+        }
+
+        finishUpdateFuture(opRes0, err0, remapTopVer0);
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings({"unchecked", "ThrowableResultOfMethodCallIgnored"})
-    @Override public void onResult(UUID nodeId, GridNearAtomicUpdateResponse res, boolean nodeErr) {
+    @Override public void onPrimaryResponse(UUID nodeId, GridNearAtomicUpdateResponse res, boolean nodeErr) {
         GridNearAtomicAbstractUpdateRequest req;
 
-        AffinityTopologyVersion remapTopVer = null;
+        AffinityTopologyVersion remapTopVer0;
 
         GridCacheReturn opRes0 = null;
         CachePartialUpdateCheckedException err0 = null;
 
-        GridFutureAdapter<?> fut0 = null;
-
         synchronized (mux) {
-            if (!res.futureVersion().equals(futVer))
+            if (futId == null || futId != res.futureId())
                 return;
 
-            if (!this.req.nodeId().equals(nodeId))
-                return;
+            req = reqState.processPrimaryResponse(nodeId, res);
 
-            req = this.req;
-
-            this.req = null;
+            if (req == null)
+                return;
 
-            boolean remapKey = !F.isEmpty(res.remapKeys());
+            boolean remapKey = res.remapTopologyVersion() != null;
 
             if (remapKey) {
-                if (mapErrTopVer == null || mapErrTopVer.compareTo(req.topologyVersion()) < 0)
-                    mapErrTopVer = req.topologyVersion();
-            }
-            else if (res.error() != null) {
-                if (res.failedKeys() != null) {
-                    if (err == null)
-                        err = new CachePartialUpdateCheckedException(
-                            "Failed to update keys (retry update if possible).");
-
-                    Collection<Object> keys = new ArrayList<>(res.failedKeys().size());
+                assert !req.topologyVersion().equals(res.remapTopologyVersion());
 
-                    for (KeyCacheObject key : res.failedKeys())
-                        keys.add(cctx.cacheObjectContext().unwrapBinaryIfNeeded(key, keepBinary, false));
+                assert remapTopVer == null : remapTopVer;
 
-                    err.add(keys, res.error(), req.topologyVersion());
-                }
+                remapTopVer = res.remapTopologyVersion();
             }
+            else if (res.error() != null)
+                onPrimaryError(req, res);
             else {
-                if (!req.fastMap() || req.hasPrimary()) {
-                    GridCacheReturn ret = res.returnValue();
-
-                    if (op == TRANSFORM) {
-                        if (ret != null) {
-                            assert ret.value() == null || ret.value() instanceof Map : ret.value();
-
-                            if (ret.value() != null) {
-                                if (opRes != null)
-                                    opRes.mergeEntryProcessResults(ret);
-                                else
-                                    opRes = ret;
-                            }
+                GridCacheReturn ret = res.returnValue();
+
+                if (op == TRANSFORM) {
+                    if (ret != null) {
+                        assert ret.value() == null || ret.value() instanceof Map : ret.value();
+
+                        if (ret.value() != null) {
+                            if (opRes != null)
+                                opRes.mergeEntryProcessResults(ret);
+                            else
+                                opRes = ret;
                         }
                     }
-                    else
-                        opRes = ret;
                 }
-            }
+                else
+                    opRes = ret;
 
-            if (remapKey) {
-                assert mapErrTopVer != null;
+                assert reqState != null;
 
-                remapTopVer = cctx.shared().exchange().topologyVersion();
+                if (!reqState.onPrimaryResponse(res, cctx))
+                    return;
             }
-            else {
-                if (err != null &&
-                    X.hasCause(err, CachePartialUpdateCheckedException.class) &&
-                    X.hasCause(err, ClusterTopologyCheckedException.class) &&
-                    storeFuture() &&
-                    --remapCnt > 0) {
-                    ClusterTopologyCheckedException topErr =
-                        X.cause(err, ClusterTopologyCheckedException.class);
-
-                    if (!(topErr instanceof ClusterTopologyServerNotFoundException)) {
-                        CachePartialUpdateCheckedException cause =
-                            X.cause(err, CachePartialUpdateCheckedException.class);
 
-                        assert cause != null && cause.topologyVersion() != null : err;
+            remapTopVer0 = onAllReceived();
 
-                        remapTopVer =
-                            new AffinityTopologyVersion(cause.topologyVersion().topologyVersion() + 1);
-
-                        err = null;
-                        updVer = null;
-                    }
-                }
-            }
-
-            if (remapTopVer == null) {
+            if (remapTopVer0 == null) {
                 err0 = err;
                 opRes0 = opRes;
             }
-            else {
-                fut0 = topCompleteFut;
-
-                topCompleteFut = null;
-
-                cctx.mvcc().removeAtomicFuture(futVer);
-
-                futVer = null;
-                topVer = AffinityTopologyVersion.ZERO;
-            }
         }
 
         if (res.error() != null && res.failedKeys() == null) {
@@ -309,55 +315,102 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
             return;
         }
 
+        if (remapTopVer0 != null) {
+            waitAndRemap(remapTopVer0);
+
+            return;
+        }
+
         if (nearEnabled && !nodeErr)
             updateNear(req, res);
 
-        if (remapTopVer != null) {
-            if (fut0 != null)
-                fut0.onDone();
+        onDone(opRes0, err0);
+    }
 
-            if (!waitTopFut) {
-                onDone(new GridCacheTryPutFailedException());
+    /**
+     * @return Non-null topology version if update should be remapped.
+     */
+    private AffinityTopologyVersion onAllReceived() {
+        assert futId != null;
 
-                return;
+        AffinityTopologyVersion remapTopVer0 = null;
+
+        if (remapTopVer == null) {
+            if (err != null &&
+                X.hasCause(err, CachePartialUpdateCheckedException.class) &&
+                X.hasCause(err, ClusterTopologyCheckedException.class) &&
+                storeFuture() &&
+                --remapCnt > 0) {
+                ClusterTopologyCheckedException topErr = X.cause(err, ClusterTopologyCheckedException.class);
+
+                if (!(topErr instanceof ClusterTopologyServerNotFoundException)) {
+                    CachePartialUpdateCheckedException cause =
+                        X.cause(err, CachePartialUpdateCheckedException.class);
+
+                    assert cause != null && cause.topologyVersion() != null : err;
+
+                    remapTopVer0 = new AffinityTopologyVersion(cause.topologyVersion().topologyVersion() + 1);
+
+                    err = null;
+                }
             }
+        }
+        else
+            remapTopVer0 = remapTopVer;
 
-            if (topLocked) {
-                CachePartialUpdateCheckedException e =
-                    new CachePartialUpdateCheckedException("Failed to update keys (retry update if possible).");
+        if (remapTopVer0 != null) {
+            cctx.mvcc().removeAtomicFuture(futId);
 
-                ClusterTopologyCheckedException cause = new ClusterTopologyCheckedException(
-                    "Failed to update keys, topology changed while execute atomic update inside transaction.");
+            reqState = null;
+            futId = null;
+            topVer = AffinityTopologyVersion.ZERO;
 
-                cause.retryReadyFuture(cctx.affinity().affinityReadyFuture(remapTopVer));
+            remapTopVer = null;
+        }
 
-                e.add(Collections.singleton(cctx.toCacheKeyObject(key)), cause);
+        return remapTopVer0;
+    }
 
-                onDone(e);
+    /**
+     * @param remapTopVer New topology version.
+     */
+    private void waitAndRemap(AffinityTopologyVersion remapTopVer) {
+        if (!waitTopFut) {
+            onDone(new GridCacheTryPutFailedException());
 
-                return;
-            }
+            return;
+        }
 
-            IgniteInternalFuture<AffinityTopologyVersion> fut =
-                cctx.shared().exchange().affinityReadyFuture(remapTopVer);
+        if (topLocked) {
+            CachePartialUpdateCheckedException e =
+                new CachePartialUpdateCheckedException("Failed to update keys (retry update if possible).");
 
-            if (fut == null)
-                fut = new GridFinishedFuture<>(remapTopVer);
+            ClusterTopologyCheckedException cause = new ClusterTopologyCheckedException(
+                "Failed to update keys, topology changed while execute atomic update inside transaction.");
 
-            fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
-                @Override public void apply(final IgniteInternalFuture<AffinityTopologyVersion> fut) {
-                    cctx.kernalContext().closure().runLocalSafe(new Runnable() {
-                        @Override public void run() {
-                            mapOnTopology();
-                        }
-                    });
-                }
-            });
+            cause.retryReadyFuture(cctx.affinity().affinityReadyFuture(remapTopVer));
+
+            e.add(Collections.singleton(cctx.toCacheKeyObject(key)), cause);
+
+            onDone(e);
 
             return;
         }
 
-        onDone(opRes0, err0);
+        IgniteInternalFuture<AffinityTopologyVersion> fut = cctx.shared().exchange().affinityReadyFuture(remapTopVer);
+
+        if (fut == null)
+            fut = new GridFinishedFuture<>(remapTopVer);
+
+        fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
+            @Override public void apply(final IgniteInternalFuture<AffinityTopologyVersion> fut) {
+                cctx.kernalContext().closure().runLocalSafe(new Runnable() {
+                    @Override public void run() {
+                        mapOnTopology();
+                    }
+                });
+            }
+        });
     }
 
     /**
@@ -369,7 +422,7 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
     private void updateNear(GridNearAtomicAbstractUpdateRequest req, GridNearAtomicUpdateResponse res) {
         assert nearEnabled;
 
-        if (res.remapKeys() != null || !req.hasPrimary())
+        if (res.remapTopologyVersion() != null)
             return;
 
         GridNearAtomicCache near = (GridNearAtomicCache)cctx.dht().near();
@@ -380,103 +433,74 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
     /** {@inheritDoc} */
     @Override protected void mapOnTopology() {
         AffinityTopologyVersion topVer;
-        GridCacheVersion futVer;
-
-        cache.topology().readLock();
 
-        try {
-            if (cache.topology().stopping()) {
-                onDone(new IgniteCheckedException("Failed to perform cache operation (cache is stopped): " +
-                    cache.name()));
+        if (cache.topology().stopping()) {
+            onDone(new IgniteCheckedException("Failed to perform cache operation (cache is stopped): " +
+                cache.name()));
 
-                return;
-            }
-
-            GridDhtTopologyFuture fut = cache.topology().topologyVersionFuture();
+            return;
+        }
 
-            if (fut.isDone()) {
-                Throwable err = fut.validateCache(cctx);
+        GridDhtTopologyFuture fut = cache.topology().topologyVersionFuture();
 
-                if (err != null) {
-                    onDone(err);
+        if (fut.isDone()) {
+            Throwable err = fut.validateCache(cctx);
 
-                    return;
-                }
-
-                topVer = fut.topologyVersion();
-
-                futVer = addAtomicFuture(topVer);
-            }
-            else {
-                if (waitTopFut) {
-                    assert !topLocked : this;
-
-                    fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
-                        @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
-                            cctx.kernalContext().closure().runLocalSafe(new Runnable() {
-                                @Override public void run() {
-                                    mapOnTopology();
-                                }
-                            });
-                        }
-                    });
-                }
-                else
-                    onDone(new GridCacheTryPutFailedException());
+            if (err != null) {
+                onDone(err);
 
                 return;
             }
-        }
-        finally {
-            cache.topology().readUnlock();
-        }
-
-        if (futVer != null)
-            map(topVer, futVer);
-    }
 
-    /** {@inheritDoc} */
-    @Override protected void map(AffinityTopologyVersion topVer, GridCacheVersion futVer) {
-        Collection<ClusterNode> topNodes = CU.affinityNodes(cctx, topVer);
-
-        if (F.isEmpty(topNodes)) {
-            onDone(new ClusterTopologyServerNotFoundException("Failed to map keys for cache (all partition nodes " +
-                "left the grid)."));
+            topVer = fut.topologyVersion();
+        }
+        else {
+            if (waitTopFut) {
+                assert !topLocked : this;
+
+                fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
+                    @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
+                        cctx.kernalContext().closure().runLocalSafe(new Runnable() {
+                            @Override public void run() {
+                                mapOnTopology();
+                            }
+                        });
+                    }
+                });
+            }
+            else
+                onDone(new GridCacheTryPutFailedException());
 
             return;
         }
 
-        GridCacheVersion updVer;
-
-        // Assign version on near node in CLOCK ordering mode even if fastMap is false.
-        if (cctx.config().getAtomicWriteOrderMode() == CLOCK) {
-            updVer = this.updVer;
-
-            if (updVer == null) {
-                updVer = futVer;
+        map(topVer);
+    }
 
-                if (log.isDebugEnabled())
-                    log.debug("Assigned fast-map version for update on near node: " + updVer);
-            }
-        }
-        else
-            updVer = null;
+    /** {@inheritDoc} */
+    @Override protected void map(AffinityTopologyVersion topVer) {
+        long futId = cctx.mvcc().atomicFutureId();
 
         Exception err = null;
-        GridNearAtomicAbstractUpdateRequest singleReq0 = null;
+        PrimaryRequestState reqState0 = null;
 
         try {
-            singleReq0 = mapSingleUpdate(topVer, futVer, updVer);
+            reqState0 = mapSingleUpdate(topVer, futId);
 
             synchronized (mux) {
-                assert this.futVer == futVer || (this.isDone() && this.error() != null);
-                assert this.topVer == topVer;
+                assert this.futId == null : this;
+                assert this.topVer == AffinityTopologyVersion.ZERO : this;
+
+                this.topVer = topVer;
+                this.futId = futId;
 
-                this.updVer = updVer;
+                reqState = reqState0;
+            }
 
-                resCnt = 0;
+            if (storeFuture() && !cctx.mvcc().addAtomicFuture(futId, this)) {
+                assert isDone();
 
-                req = singleReq0;
+                return;
             }
         }
         catch (Exception e) {
@@ -490,43 +514,80 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
         }
 
         // Optimize mapping for single key.
-        mapSingle(singleReq0.nodeId(), singleReq0);
+        sendSingleRequest(reqState0.req.nodeId(), reqState0.req);
+
+        if (syncMode == FULL_ASYNC) {
+            onDone(new GridCacheReturn(cctx, true, true, null, true));
+
+            return;
+        }
+
+        if (reqState0.req.initMappingLocally() && (cctx.discovery().topologyVersion() != topVer.topologyVersion()))
+            checkDhtNodes(futId);
     }
 
     /**
-     * @return Future version.
+     * @param futId
+     * @return
      */
-    private GridCacheVersion onFutureDone() {
-        GridCacheVersion ver0;
+    private boolean checkDhtNodes(Long futId) {
+        GridCacheReturn opRes0 = null;
+        CachePartialUpdateCheckedException err0 = null;
+        AffinityTopologyVersion remapTopVer0 = null;
 
-        GridFutureAdapter<Void> fut0;
+        GridNearAtomicCheckUpdateRequest checkReq = null;
 
         synchronized (mux) {
-            fut0 = topCompleteFut;
+            if (this.futId == null || !this.futId.equals(futId))
+                return false;
 
-            topCompleteFut = null;
+            assert reqState != null;
 
-            ver0 = futVer;
+            DhtLeftResult res = reqState.checkDhtNodes(cctx);
 
-            futVer = null;
+            if (res == DhtLeftResult.DONE) {
+                opRes0 = opRes;
+                err0 = err;
+                remapTopVer0 = onAllReceived();
+            }
+            else if (res == DhtLeftResult.ALL_RCVD_CHECK_PRIMARY){
+                checkReq = new GridNearAtomicCheckUpdateRequest(reqState.req);
+            }
+            else
+                return true;
         }
 
-        if (fut0 != null)
-            fut0.onDone();
+        if (checkReq != null)
+            sendCheckUpdateRequest(checkReq);
+        else
+            finishUpdateFuture(opRes0, err0, remapTopVer0);
+
+        return false;
+    }
+
+    /**
+     * @return Future ID.
+     */
+    private Long onFutureDone() {
+        Long id0;
+
+        synchronized (mux) {
+            id0 = futId;
 
-        return ver0;
+            futId = null;
+        }
+
+        return id0;
     }
 
     /**
      * @param topVer Topology version.
-     * @param futVer Future version.
-     * @param updVer Update version.
+     * @param futId Future ID.
      * @return Request.
      * @throws Exception If failed.
      */
-    private GridNearAtomicAbstractUpdateRequest mapSingleUpdate(AffinityTopologyVersion topVer,
-        GridCacheVersion futVer,
-        @Nullable GridCacheVersion updVer) throws Exception {
+    private PrimaryRequestState mapSingleUpdate(AffinityTopologyVersion topVer, long futId)
+        throws Exception {
         if (key == null)
             throw new NullPointerException("Null key.");
 
@@ -542,22 +603,27 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
         else
             val = EntryProcessorResourceInjectorProxy.wrap(cctx.kernalContext(), (EntryProcessor)val);
 
-        ClusterNode primary = cctx.affinity().primaryByKey(cacheKey, topVer);
+        boolean mappingKnown = cctx.topology().rebalanceFinished(topVer) &&
+            !cctx.discovery().hasNearCache(cctx.cacheId(), topVer);
+
+        List<ClusterNode> nodes = cctx.affinity().nodesByKey(cacheKey, topVer);
 
-        if (primary == null)
+        if (F.isEmpty(nodes))
             throw new ClusterTopologyServerNotFoundException("Failed to map keys for cache (all partition nodes " +
                 "left the grid).");
 
+        ClusterNode primary = nodes.get(0);
+
+        boolean needPrimaryRes = !mappingKnown || primary.isLocal() || nodes.size() == 1;
+
         GridNearAtomicAbstractUpdateRequest req;
 
-        if (canUseSingleRequest(primary)) {
+        if (canUseSingleRequest()) {
             if (op == TRANSFORM) {
                 req = new GridNearAtomicSingleUpdateInvokeRequest(
                     cctx.cacheId(),
                     primary.id(),
-                    futVer,
-                    false,
-                    updVer,
+                    futId,
                     topVer,
                     topLocked,
                     syncMode,
@@ -566,9 +632,9 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
                     invokeArgs,
                     subjId,
                     taskNameHash,
+                    needPrimaryRes,
                     skipStore,
                     keepBinary,
-                    cctx.kernalContext().clientNode(),
                     cctx.deploymentEnabled());
             }
             else {
@@ -576,9 +642,7 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
                     req = new GridNearAtomicSingleUpdateRequest(
                         cctx.cacheId(),
                         primary.id(),
-                        futVer,
-                        false,
-                        updVer,
+                        futId,
                         topVer,
                         topLocked,
                         syncMode,
@@ -586,18 +650,16 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
                         retval,
                         subjId,
                         taskNameHash,
+                        needPrimaryRes,
                         skipStore,
                         keepBinary,
-                        cctx.kernalContext().clientNode(),
                         cctx.deploymentEnabled());
                 }
                 else {
                     req = new GridNearAtomicSingleUpdateFilterRequest(
                         cctx.cacheId(),
                         primary.id(),
-                        futVer,
-                        false,
-                        updVer,
+                        futId,
                         topVer,
                         topLocked,
                         syncMode,
@@ -606,9 +668,9 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
                         filter,
                         subjId,
                         taskNameHash,
+                        needPrimaryRes,
                         skipStore,
                         keepBinary,
-                        cctx.kernalContext().clientNode(),
                         cctx.deploymentEnabled());
                 }
             }
@@ -617,9 +679,7 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
             req = new GridNearAtomicFullUpdateRequest(
                 cctx.cacheId(),
                 primary.id(),
-                futVer,
-                false,
-                updVer,
+                futId,
                 topVer,
                 topLocked,
                 syncMode,
@@ -630,9 +690,9 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
                 filter,
                 subjId,
                 taskNameHash,
+                needPrimaryRes,
                 skipStore,
                 keepBinary,
-                cctx.kernalContext().clientNode(),
                 cctx.deploymentEnabled(),
                 1);
         }
@@ -641,18 +701,39 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
             val,
             CU.TTL_NOT_CHANGED,
             CU.EXPIRE_TIME_CALCULATE,
-            null,
-            true);
+            null);
+
+        return new PrimaryRequestState(req, nodes, true);
+    }
+
+    /**
+     * @param opRes Operation result.
+     * @param err Operation error.
+     * @param remapTopVer Not-null topology version if need remap update.
+     */
+    private void finishUpdateFuture(GridCacheReturn opRes,
+        CachePartialUpdateCheckedException err,
+        @Nullable AffinityTopologyVersion remapTopVer) {
+        if (remapTopVer != null) {
+            waitAndRemap(remapTopVer);
+
+            return;
+        }
+
+        if (nearEnabled) {
+            assert reqState.req.response() != null;
+
+            updateNear(reqState.req, reqState.req.response());
+        }
 
-        return req;
+        onDone(opRes, err);
     }
 
     /**
-     * @param node Target node
      * @return {@code True} can use 'single' update requests.
      */
-    private boolean canUseSingleRequest(ClusterNode node) {
-        return expiryPlc == null && node != null && node.version().compareToIgnoreTimestamp(SINGLE_UPDATE_REQUEST) >= 0;
+    private boolean canUseSingleRequest() {
+        return expiryPlc == null;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
index 6582063..f8b3984 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
@@ -76,9 +76,7 @@ public class GridNearAtomicSingleUpdateInvokeRequest extends GridNearAtomicSingl
      *
      * @param cacheId Cache ID.
      * @param nodeId Node ID.
-     * @param futVer Future version.
-     * @param fastMap Fast map scheme flag.
-     * @param updateVer Update version set if fast map is performed.
+     * @param futId Future ID.
      * @param topVer Topology version.
      * @param topLocked Topology locked flag.
      * @param syncMode Synchronization mode.
@@ -89,15 +87,12 @@ public class GridNearAtomicSingleUpdateInvokeRequest extends GridNearAtomicSingl
      * @param taskNameHash Task name hash code.
      * @param skipStore Skip write-through to a persistent storage.
      * @param keepBinary Keep binary flag.
-     * @param clientReq Client node request flag.
      * @param addDepInfo Deployment info flag.
      */
     GridNearAtomicSingleUpdateInvokeRequest(
         int cacheId,
         UUID nodeId,
-        GridCacheVersion futVer,
-        boolean fastMap,
-        @Nullable GridCacheVersion updateVer,
+        long futId,
         @NotNull AffinityTopologyVersion topVer,
         boolean topLocked,
         CacheWriteSynchronizationMode syncMode,
@@ -106,17 +101,15 @@ public class GridNearAtomicSingleUpdateInvokeRequest extends GridNearAtomicSingl
         @Nullable Object[] invokeArgs,
         @Nullable UUID subjId,
         int taskNameHash,
+        boolean needPrimaryRes,
         boolean skipStore,
         boolean keepBinary,
-        boolean clientReq,
         boolean addDepInfo
     ) {
         super(
             cacheId,
             nodeId,
-            futVer,
-            fastMap,
-            updateVer,
+            futId,
             topVer,
             topLocked,
             syncMode,
@@ -124,14 +117,15 @@ public class GridNearAtomicSingleUpdateInvokeRequest extends GridNearAtomicSingl
             retval,
             subjId,
             taskNameHash,
+            needPrimaryRes,
             skipStore,
             keepBinary,
-            clientReq,
             addDepInfo
         );
-        this.invokeArgs = invokeArgs;
 
         assert op == TRANSFORM : op;
+
+        this.invokeArgs = invokeArgs;
     }
 
     /**
@@ -140,14 +134,12 @@ public class GridNearAtomicSingleUpdateInvokeRequest extends GridNearAtomicSingl
      * @param conflictTtl Conflict TTL (optional).
      * @param conflictExpireTime Conflict expire time (optional).
      * @param conflictVer Conflict version (optional).
-     * @param primary If given key is primary on this mapping.
      */
     @Override public void addUpdateEntry(KeyCacheObject key,
         @Nullable Object val,
         long conflictTtl,
         long conflictExpireTime,
-        @Nullable GridCacheVersion conflictVer,
-        boolean primary) {
+        @Nullable GridCacheVersion conflictVer) {
         assert conflictTtl < 0 : conflictTtl;
         assert conflictExpireTime < 0 : conflictExpireTime;
         assert conflictVer == null : conflictVer;
@@ -156,9 +148,6 @@ public class GridNearAtomicSingleUpdateInvokeRequest extends GridNearAtomicSingl
         entryProcessor = (EntryProcessor<Object, Object, Object>)val;
 
         this.key = key;
-        partId = key.partition();
-
-        hasPrimary(hasPrimary() | primary);
     }
 
     /** {@inheritDoc} */
@@ -246,13 +235,13 @@ public class GridNearAtomicSingleUpdateInvokeRequest extends GridNearAtomicSingl
         }
 
         switch (writer.state()) {
-            case 14:
+            case 12:
                 if (!writer.writeByteArray("entryProcessorBytes", entryProcessorBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 15:
+            case 13:
                 if (!writer.writeObjectArray("invokeArgsBytes", invokeArgsBytes, MessageCollectionItemType.BYTE_ARR))
                     return false;
 
@@ -274,7 +263,7 @@ public class GridNearAtomicSingleUpdateInvokeRequest extends GridNearAtomicSingl
             return false;
 
         switch (reader.state()) {
-            case 14:
+            case 12:
                 entryProcessorBytes = reader.readByteArray("entryProcessorBytes");
 
                 if (!reader.isLastRead())
@@ -282,7 +271,7 @@ public class GridNearAtomicSingleUpdateInvokeRequest extends GridNearAtomicSingl
 
                 reader.incrementState();
 
-            case 15:
+            case 13:
                 invokeArgsBytes = reader.readObjectArray("invokeArgsBytes", MessageCollectionItemType.BYTE_ARR, byte[].class);
 
                 if (!reader.isLastRead())
@@ -297,7 +286,7 @@ public class GridNearAtomicSingleUpdateInvokeRequest extends GridNearAtomicSingl
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 16;
+        return 14;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
index c3e9fbe..b9a1fc6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
@@ -57,9 +57,6 @@ public class GridNearAtomicSingleUpdateRequest extends GridNearAtomicAbstractSin
     /** Value to update. */
     protected CacheObject val;
 
-    /** Partition of key. */
-    protected int partId;
-
     /**
      * Empty constructor required by {@link Externalizable}.
      */
@@ -72,9 +69,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridNearAtomicAbstractSin
      *
      * @param cacheId Cache ID.
      * @param nodeId Node ID.
-     * @param futVer Future version.
-     * @param fastMap Fast map scheme flag.
-     * @param updateVer Update version set if fast map is performed.
+     * @param futId Future ID.
      * @param topVer Topology version.
      * @param topLocked Topology locked flag.
      * @param syncMode Synchronization mode.
@@ -84,15 +79,12 @@ public class GridNearAtomicSingleUpdateRequest extends GridNearAtomicAbstractSin
      * @param taskNameHash Task name hash code.
      * @param skipStore Skip write-through to a persistent storage.
      * @param keepBinary Keep binary flag.
-     * @param clientReq Client node request flag.
      * @param addDepInfo Deployment info flag.
      */
     GridNearAtomicSingleUpdateRequest(
         int cacheId,
         UUID nodeId,
-        GridCacheVersion futVer,
-        boolean fastMap,
-        @Nullable GridCacheVersion updateVer,
+        long futId,
         @NotNull AffinityTopologyVersion topVer,
         boolean topLocked,
         CacheWriteSynchronizationMode syncMode,
@@ -100,17 +92,14 @@ public class GridNearAtomicSingleUpdateRequest extends GridNearAtomicAbstractSin
         boolean retval,
         @Nullable UUID subjId,
         int taskNameHash,
+        boolean needPrimaryRes,
         boolean skipStore,
         boolean keepBinary,
-        boolean clientReq,
         boolean addDepInfo
     ) {
-        super(
-            cacheId,
+        super(cacheId,
             nodeId,
-            futVer,
-            fastMap,
-            updateVer,
+            futId,
             topVer,
             topLocked,
             syncMode,
@@ -118,16 +107,17 @@ public class GridNearAtomicSingleUpdateRequest extends GridNearAtomicAbstractSin
             retval,
             subjId,
             taskNameHash,
+            needPrimaryRes,
             skipStore,
             keepBinary,
-            clientReq,
-            addDepInfo
-        );
+            addDepInfo);
     }
 
     /** {@inheritDoc} */
     @Override public int partition() {
-        return partId;
+        assert key != null;
+
+        return key.partition();
     }
 
     /**
@@ -136,14 +126,12 @@ public class GridNearAtomicSingleUpdateRequest extends GridNearAtomicAbstractSin
      * @param conflictTtl Conflict TTL (optional).
      * @param conflictExpireTime Conflict expire time (optional).
      * @param conflictVer Conflict version (optional).
-     * @param primary If given key is primary on this mapping.
      */
     @Override public void addUpdateEntry(KeyCacheObject key,
         @Nullable Object val,
         long conflictTtl,
         long conflictExpireTime,
-        @Nullable GridCacheVersion conflictVer,
-        boolean primary) {
+        @Nullable GridCacheVersion conflictVer) {
         assert op != TRANSFORM;
         assert val != null || op == DELETE;
         assert conflictTtl < 0 : conflictTtl;
@@ -151,19 +139,18 @@ public class GridNearAtomicSingleUpdateRequest extends GridNearAtomicAbstractSin
         assert conflictVer == null : conflictVer;
 
         this.key = key;
-        partId = key.partition();
 
         if (val != null) {
             assert val instanceof CacheObject : val;
 
             this.val = (CacheObject)val;
         }
-
-        hasPrimary(hasPrimary() | primary);
     }
 
     /** {@inheritDoc} */
     @Override public int size() {
+        assert key != null;
+
         return key == null ? 0 : 1;
     }
 
@@ -253,8 +240,6 @@ public class GridNearAtomicSingleUpdateRequest extends GridNearAtomicAbstractSin
 
         if (val != null)
             val.finishUnmarshal(cctx.cacheObjectContext(), ldr);
-
-        key.partition(partId);
     }
 
     /** {@inheritDoc} */
@@ -272,19 +257,13 @@ public class GridNearAtomicSingleUpdateRequest extends GridNearAtomicAbstractSin
         }
 
         switch (writer.state()) {
-            case 11:
+            case 10:
                 if (!writer.writeMessage("key", key))
                     return false;
 
                 writer.incrementState();
 
-            case 12:
-                if (!writer.writeInt("partId", partId))
-                    return false;
-
-                writer.incrementState();
-
-            case 13:
+            case 11:
                 if (!writer.writeMessage("val", val))
                     return false;
 
@@ -306,7 +285,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridNearAtomicAbstractSin
             return false;
 
         switch (reader.state()) {
-            case 11:
+            case 10:
                 key = reader.readMessage("key");
 
                 if (!reader.isLastRead())
@@ -314,15 +293,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridNearAtomicAbstractSin
 
                 reader.incrementState();
 
-            case 12:
-                partId = reader.readInt("partId");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 13:
+            case 11:
                 val = reader.readMessage("val");
 
                 if (!reader.isLastRead())
@@ -350,7 +321,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridNearAtomicAbstractSin
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 14;
+        return 12;
     }
 
     /** {@inheritDoc} */


[2/8] ignite git commit: ignite-4705 Atomic cache protocol change: notify client node from backups

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/IpcToNioAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/IpcToNioAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/IpcToNioAdapter.java
index d108b56..7af6139 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/IpcToNioAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/IpcToNioAdapter.java
@@ -24,6 +24,7 @@ import java.nio.ByteOrder;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.util.nio.GridNioFilter;
 import org.apache.ignite.internal.util.nio.GridNioFilterAdapter;
@@ -36,6 +37,7 @@ import org.apache.ignite.internal.util.nio.GridNioServerListener;
 import org.apache.ignite.internal.util.nio.GridNioSession;
 import org.apache.ignite.internal.util.nio.GridNioSessionImpl;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.plugin.extensions.communication.Message;
 
 /**
@@ -201,7 +203,10 @@ public class IpcToNioAdapter<T> {
         }
 
         /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) {
+        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses,
+            Object msg,
+            boolean fut,
+            IgniteInClosure<IgniteException> ackC) {
             assert ses == IpcToNioAdapter.this.ses;
 
             return send((Message)msg);

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
index 7987d3d..f110cf2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
@@ -19,9 +19,11 @@ package org.apache.ignite.internal.util.nio;
 
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
 
 /**
  * Verifies that first bytes received in accepted (incoming)
@@ -73,9 +75,10 @@ public class GridConnectionBytesVerifyFilter extends GridNioFilterAdapter {
     @Override public GridNioFuture<?> onSessionWrite(
         GridNioSession ses,
         Object msg,
-        boolean fut
+        boolean fut,
+        IgniteInClosure<IgniteException> ackC
     ) throws IgniteCheckedException {
-        return proceedSessionWrite(ses, msg, fut);
+        return proceedSessionWrite(ses, msg, fut, ackC);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioAsyncNotifyFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioAsyncNotifyFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioAsyncNotifyFilter.java
index 5d90cdb..d55bc54 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioAsyncNotifyFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioAsyncNotifyFilter.java
@@ -19,10 +19,12 @@ package org.apache.ignite.internal.util.nio;
 
 import java.util.concurrent.Executor;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.internal.util.worker.GridWorkerPool;
+import org.apache.ignite.lang.IgniteInClosure;
 
 /**
  * Enables multithreaded notification of session opened, message received and session closed events.
@@ -110,9 +112,10 @@ public class GridNioAsyncNotifyFilter extends GridNioFilterAdapter {
     @Override public GridNioFuture<?> onSessionWrite(
         GridNioSession ses,
         Object msg,
-        boolean fut
+        boolean fut,
+        IgniteInClosure<IgniteException> ackC
     ) throws IgniteCheckedException {
-        return proceedSessionWrite(ses, msg, fut);
+        return proceedSessionWrite(ses, msg, fut, ackC);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
index 343e625..b81086a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
@@ -20,10 +20,12 @@ package org.apache.ignite.internal.util.nio;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteInClosure;
 
 /**
  * Filter that transforms byte buffers to user-defined objects and vice-versa
@@ -82,16 +84,17 @@ public class GridNioCodecFilter extends GridNioFilterAdapter {
     @Override public GridNioFuture<?> onSessionWrite(
         GridNioSession ses,
         Object msg,
-        boolean fut
+        boolean fut,
+        IgniteInClosure<IgniteException> ackC
     ) throws IgniteCheckedException {
         // No encoding needed in direct mode.
         if (directMode)
-            return proceedSessionWrite(ses, msg, fut);
+            return proceedSessionWrite(ses, msg, fut, ackC);
 
         try {
             ByteBuffer res = parser.encode(ses, msg);
 
-            return proceedSessionWrite(ses, res, fut);
+            return proceedSessionWrite(ses, res, fut, ackC);
         }
         catch (IOException e) {
             throw new GridNioException(e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioEmbeddedFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioEmbeddedFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioEmbeddedFuture.java
index be77d39..eab4909 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioEmbeddedFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioEmbeddedFuture.java
@@ -31,6 +31,13 @@ public class GridNioEmbeddedFuture<R> extends GridNioFutureImpl<R> {
     private static final long serialVersionUID = 0L;
 
     /**
+     *
+     */
+    public GridNioEmbeddedFuture() {
+        super(null);
+    }
+
+    /**
      * Callback to notify that future is finished.
      * This method must delegate to {@link #onDone(GridNioFuture, Throwable)} method.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilter.java
index f7928c4..9163a4f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilter.java
@@ -18,6 +18,8 @@
 package org.apache.ignite.internal.util.nio;
 
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.lang.IgniteInClosure;
 
 /**
  * This interface defines the general element in transformation chain between the nio server and
@@ -106,13 +108,15 @@ public interface GridNioFilter {
      * @param ses Session instance.
      * @param msg Message to send.
      * @param fut {@code True} if write future should be created.
+     * @param ackC Closure invoked when message ACK is received.
      * @return Write future or {@code null}.
      * @throws IgniteCheckedException If filter is not in chain or GridNioException occurred in the underlying filter.
      */
     public GridNioFuture<?> proceedSessionWrite(
         GridNioSession ses,
         Object msg,
-        boolean fut
+        boolean fut,
+        IgniteInClosure<IgniteException> ackC
     ) throws IgniteCheckedException;
 
     /**
@@ -155,10 +159,14 @@ public interface GridNioFilter {
      * @param ses Session on which message should be written.
      * @param msg Message being written.
      * @param fut {@code True} if write future should be created.
+     * @param ackC Closure invoked when message ACK is received.
      * @return Write future or {@code null}.
      * @throws GridNioException If GridNioException occurred while handling event.
      */
-    public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) throws IgniteCheckedException;
+    public GridNioFuture<?> onSessionWrite(GridNioSession ses,
+        Object msg,
+        boolean fut,
+        IgniteInClosure<IgniteException> ackC) throws IgniteCheckedException;
 
     /**
      * Invoked when a new messages received.

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterAdapter.java
index 58ddae5..4ede4b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterAdapter.java
@@ -18,6 +18,8 @@
 package org.apache.ignite.internal.util.nio;
 
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.lang.IgniteInClosure;
 
 /**
  * Class that defines the piece for application-to-network and vice-versa data conversions
@@ -111,11 +113,12 @@ public abstract class GridNioFilterAdapter implements GridNioFilter {
     @Override public GridNioFuture<?> proceedSessionWrite(
         GridNioSession ses,
         Object msg,
-        boolean fut
+        boolean fut,
+        IgniteInClosure<IgniteException> ackC
     ) throws IgniteCheckedException {
         checkNext();
 
-        return nextFilter.onSessionWrite(ses, msg, fut);
+        return nextFilter.onSessionWrite(ses, msg, fut, ackC);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
index 8cc690b..ec59020 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
@@ -18,8 +18,10 @@
 package org.apache.ignite.internal.util.nio;
 
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.util.typedef.internal.LT;
+import org.apache.ignite.lang.IgniteInClosure;
 
 /**
  * Filter chain implementation for nio server filters.
@@ -184,9 +186,10 @@ public class GridNioFilterChain<T> extends GridNioFilterAdapter {
     @Override public GridNioFuture<?> onSessionWrite(
         GridNioSession ses,
         Object msg,
-        boolean fut
+        boolean fut,
+        IgniteInClosure<IgniteException> ackC
     ) throws IgniteCheckedException {
-        return tail.onSessionWrite(ses, msg, fut);
+        return tail.onSessionWrite(ses, msg, fut, ackC);
     }
 
     /**
@@ -259,9 +262,11 @@ public class GridNioFilterChain<T> extends GridNioFilterAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut)
-            throws IgniteCheckedException {
-            return proceedSessionWrite(ses, msg, fut);
+        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses,
+            Object msg,
+            boolean fut,
+            IgniteInClosure<IgniteException> ackC) throws IgniteCheckedException {
+            return proceedSessionWrite(ses, msg, fut, ackC);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFinishedFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFinishedFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFinishedFuture.java
index 3d18ab7..2835a22 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFinishedFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFinishedFuture.java
@@ -59,11 +59,6 @@ public class GridNioFinishedFuture<R> extends GridFinishedFuture<R> implements G
     }
 
     /** {@inheritDoc} */
-    @Override public void ackClosure(IgniteInClosure<IgniteException> closure) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
     @Override public void onAckReceived() {
         // No-op.
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java
index 6c0c9c6..4d1fee6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java
@@ -43,13 +43,6 @@ public interface GridNioFuture<R> extends IgniteInternalFuture<R> {
     public boolean skipRecovery();
 
     /**
-     * Sets ack closure which will be applied when ack received.
-     *
-     * @param c Ack closure.
-     */
-    public void ackClosure(IgniteInClosure<IgniteException> c);
-
-    /**
      * The method will be called when ack received.
      */
     public void onAckReceived();

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFutureImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFutureImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFutureImpl.java
index fe97039..6a94a54 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFutureImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFutureImpl.java
@@ -30,10 +30,17 @@ public class GridNioFutureImpl<R> extends GridFutureAdapter<R> implements GridNi
     private static final long serialVersionUID = 0L;
 
     /** */
-    protected boolean msgThread;
+    private boolean msgThread;
 
     /** */
-    protected IgniteInClosure<IgniteException> ackClosure;
+    protected final IgniteInClosure<IgniteException> ackC;
+
+    /**
+     * @param ackC Ack closure.
+     */
+    public GridNioFutureImpl(IgniteInClosure<IgniteException> ackC) {
+        this.ackC = ackC;
+    }
 
     /** {@inheritDoc} */
     @Override public void messageThread(boolean msgThread) {
@@ -51,18 +58,13 @@ public class GridNioFutureImpl<R> extends GridFutureAdapter<R> implements GridNi
     }
 
     /** {@inheritDoc} */
-    @Override public void ackClosure(IgniteInClosure<IgniteException> closure) {
-        ackClosure = closure;
-    }
-
-    /** {@inheritDoc} */
     @Override public void onAckReceived() {
         // No-op.
     }
 
     /** {@inheritDoc} */
     @Override public IgniteInClosure<IgniteException> ackClosure() {
-        return ackClosure;
+        return ackC;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index fefdf15..7f25e40 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -74,7 +74,6 @@ import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.Nullable;
 import sun.nio.ch.DirectBuffer;
 
-import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.ACK_CLOSURE;
 import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.MSG_WRITER;
 import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.NIO_OPERATION;
 
@@ -481,22 +480,26 @@ public class GridNioServer<T> {
      * @param ses Session.
      * @param msg Message.
      * @param createFut {@code True} if future should be created.
+     * @param ackC Closure invoked when message ACK is received.
      * @return Future for operation.
      */
-    GridNioFuture<?> send(GridNioSession ses, ByteBuffer msg, boolean createFut) throws IgniteCheckedException {
+    GridNioFuture<?> send(GridNioSession ses,
+        ByteBuffer msg,
+        boolean createFut,
+        IgniteInClosure<IgniteException> ackC) throws IgniteCheckedException {
         assert ses instanceof GridSelectorNioSessionImpl : ses;
 
         GridSelectorNioSessionImpl impl = (GridSelectorNioSessionImpl)ses;
 
         if (createFut) {
-            NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg);
+            NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg, ackC);
 
             send0(impl, fut, false);
 
             return fut;
         }
         else {
-            SessionWriteRequest req = new WriteRequestImpl(ses, msg, true);
+            SessionWriteRequest req = new WriteRequestImpl(ses, msg, true, ackC);
 
             send0(impl, req, false);
 
@@ -508,23 +511,27 @@ public class GridNioServer<T> {
      * @param ses Session.
      * @param msg Message.
      * @param createFut {@code True} if future should be created.
+     * @param ackC Closure invoked when message ACK is received.
      * @return Future for operation.
      */
-    GridNioFuture<?> send(GridNioSession ses, Message msg, boolean createFut) throws IgniteCheckedException {
+    GridNioFuture<?> send(GridNioSession ses,
+        Message msg,
+        boolean createFut,
+        IgniteInClosure<IgniteException> ackC) throws IgniteCheckedException {
         assert ses instanceof GridSelectorNioSessionImpl;
 
         GridSelectorNioSessionImpl impl = (GridSelectorNioSessionImpl)ses;
 
         if (createFut) {
             NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg,
-                skipRecoveryPred.apply(msg));
+                skipRecoveryPred.apply(msg), ackC);
 
             send0(impl, fut, false);
 
             return fut;
         }
         else {
-            SessionWriteRequest req = new WriteRequestImpl(ses, msg, skipRecoveryPred.apply(msg));
+            SessionWriteRequest req = new WriteRequestImpl(ses, msg, skipRecoveryPred.apply(msg), ackC);
 
             send0(impl, req, false);
 
@@ -544,11 +551,6 @@ public class GridNioServer<T> {
 
         int msgCnt = sys ? ses.offerSystemFuture(req) : ses.offerFuture(req);
 
-        IgniteInClosure<IgniteException> ackC;
-
-        if (!sys && (ackC = ses.removeMeta(ACK_CLOSURE.ordinal())) != null)
-            req.ackClosure(ackC);
-
         if (ses.closed()) {
             if (ses.removeFuture(req)) {
                 IOException err = new IOException("Failed to send message (connection was closed): " + ses);
@@ -597,8 +599,11 @@ public class GridNioServer<T> {
         GridSelectorNioSessionImpl impl = (GridSelectorNioSessionImpl)ses;
 
         if (lsnr != null) {
-            NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg,
-                skipRecoveryPred.apply(msg));
+            NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl,
+                NioOperation.REQUIRE_WRITE,
+                msg,
+                skipRecoveryPred.apply(msg),
+                null);
 
             fut.listen(lsnr);
 
@@ -2597,11 +2602,6 @@ public class GridNioServer<T> {
         }
 
         /** {@inheritDoc} */
-        @Override public void ackClosure(IgniteInClosure<IgniteException> c) {
-            throw new UnsupportedOperationException();
-        }
-
-        /** {@inheritDoc} */
         @Override public void onAckReceived() {
             throw new UnsupportedOperationException();
         }
@@ -2664,17 +2664,22 @@ public class GridNioServer<T> {
         private final boolean skipRecovery;
 
         /** */
-        private IgniteInClosure<IgniteException> ackC;
+        private final IgniteInClosure<IgniteException> ackC;
 
         /**
          * @param ses Session.
          * @param msg Message.
          * @param skipRecovery Skip recovery flag.
+         * @param ackC Closure invoked when message ACK is received.
          */
-        WriteRequestImpl(GridNioSession ses, Object msg, boolean skipRecovery) {
+        WriteRequestImpl(GridNioSession ses,
+            Object msg,
+            boolean skipRecovery,
+            IgniteInClosure<IgniteException> ackC) {
             this.ses = ses;
             this.msg = msg;
             this.skipRecovery = skipRecovery;
+            this.ackC = ackC;
         }
 
         /** {@inheritDoc} */
@@ -2693,11 +2698,6 @@ public class GridNioServer<T> {
         }
 
         /** {@inheritDoc} */
-        @Override public void ackClosure(IgniteInClosure<IgniteException> c) {
-            ackC = c;
-        }
-
-        /** {@inheritDoc} */
         @Override public void onAckReceived() {
             assert msg instanceof Message;
 
@@ -2798,6 +2798,8 @@ public class GridNioServer<T> {
             boolean accepted,
             @Nullable Map<Integer, ?> meta
         ) {
+            super(null);
+
             op = NioOperation.REGISTER;
 
             this.sockCh = sockCh;
@@ -2812,6 +2814,8 @@ public class GridNioServer<T> {
          * @param op Requested operation.
          */
         NioOperationFuture(GridSelectorNioSessionImpl ses, NioOperation op) {
+            super(null);
+
             assert ses != null || op == NioOperation.DUMP_STATS : "Invalid params [ses=" + ses + ", op=" + op + ']';
             assert op != null;
             assert op != NioOperation.REGISTER;
@@ -2826,8 +2830,14 @@ public class GridNioServer<T> {
          * @param ses Session to change.
          * @param op Requested operation.
          * @param msg Message.
+         * @param ackC Closure invoked when message ACK is received.
          */
-        NioOperationFuture(GridSelectorNioSessionImpl ses, NioOperation op, Object msg) {
+        NioOperationFuture(GridSelectorNioSessionImpl ses,
+            NioOperation op,
+            Object msg,
+            IgniteInClosure<IgniteException> ackC) {
+            super(ackC);
+
             assert ses != null;
             assert op != null;
             assert op != NioOperation.REGISTER;
@@ -2845,9 +2855,15 @@ public class GridNioServer<T> {
          * @param op Requested operation.
          * @param commMsg Direct message.
          * @param skipRecovery Skip recovery flag.
+         * @param ackC Closure invoked when message ACK is received.
          */
-        NioOperationFuture(GridSelectorNioSessionImpl ses, NioOperation op,
-            Message commMsg, boolean skipRecovery) {
+        NioOperationFuture(GridSelectorNioSessionImpl ses,
+            NioOperation op,
+            Message commMsg,
+            boolean skipRecovery,
+            IgniteInClosure<IgniteException> ackC) {
+            super(ackC);
+
             assert ses != null;
             assert op != null;
             assert op != NioOperation.REGISTER;
@@ -3013,7 +3029,10 @@ public class GridNioServer<T> {
         }
 
         /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) throws IgniteCheckedException {
+        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses,
+            Object msg,
+            boolean fut,
+            IgniteInClosure<IgniteException> ackC) throws IgniteCheckedException {
             if (directMode) {
                 boolean sslSys = sslFilter != null && msg instanceof ByteBuffer;
 
@@ -3032,10 +3051,10 @@ public class GridNioServer<T> {
                     return null;
                 }
                 else
-                    return send(ses, (Message)msg, fut);
+                    return send(ses, (Message)msg, fut, ackC);
             }
             else
-                return send(ses, (ByteBuffer)msg, fut);
+                return send(ses, (ByteBuffer)msg, fut, ackC);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java
index c1b60ab..21eabf3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java
@@ -19,6 +19,8 @@ package org.apache.ignite.internal.util.nio;
 
 import java.net.InetSocketAddress;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.lang.IgniteInClosure;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -107,8 +109,11 @@ public interface GridNioSession {
 
     /**
      * @param msg Message to be sent.
+     * @param ackC Optional closure invoked when ack for message is received.
+     * @throws IgniteCheckedException If failed.
      */
-    public void sendNoFuture(Object msg) throws IgniteCheckedException;
+    public void sendNoFuture(Object msg, @Nullable IgniteInClosure<IgniteException> ackC)
+        throws IgniteCheckedException;
 
     /**
      * Gets metadata associated with specified key.

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java
index 7424531..98a22d6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java
@@ -20,8 +20,10 @@ package org.apache.ignite.internal.util.nio;
 import java.net.InetSocketAddress;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.MAX_KEYS_CNT;
@@ -105,7 +107,7 @@ public class GridNioSessionImpl implements GridNioSession {
         try {
             resetSendScheduleTime();
 
-            return chain().onSessionWrite(this, msg, true);
+            return chain().onSessionWrite(this, msg, true, null);
         }
         catch (IgniteCheckedException e) {
             close();
@@ -115,9 +117,10 @@ public class GridNioSessionImpl implements GridNioSession {
     }
 
     /** {@inheritDoc} */
-    @Override public void sendNoFuture(Object msg) throws IgniteCheckedException {
+    @Override public void sendNoFuture(Object msg, IgniteInClosure<IgniteException> ackC)
+        throws IgniteCheckedException {
         try {
-            chain().onSessionWrite(this, msg, false);
+            chain().onSessionWrite(this, msg, false, ackC);
         }
         catch (IgniteCheckedException e) {
             close();

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionMetaKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionMetaKey.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionMetaKey.java
index bdb3a29..5385430 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionMetaKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionMetaKey.java
@@ -42,10 +42,7 @@ public enum GridNioSessionMetaKey {
     MARSHALLER_ID,
 
     /** Message writer. */
-    MSG_WRITER,
-
-    /** Ack closure. */
-    ACK_CLOSURE;
+    MSG_WRITER;
 
     /** Maximum count of NIO session keys in system. */
     public static final int MAX_KEYS_CNT = 64;

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
index d941bae..ab9b2eb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
@@ -123,7 +123,7 @@ public class GridShmemCommunicationClient extends GridAbstractCommunicationClien
 
     /** {@inheritDoc} */
     @Override public synchronized boolean sendMessage(UUID nodeId, Message msg,
-        IgniteInClosure<IgniteException> closure) throws IgniteCheckedException {
+        IgniteInClosure<IgniteException> c) throws IgniteCheckedException {
         assert nodeId != null;
 
         if (closed())
@@ -142,8 +142,8 @@ public class GridShmemCommunicationClient extends GridAbstractCommunicationClien
 
         markUsed();
 
-        if (closure != null)
-            closure.apply(null);
+        if (c != null)
+            c.apply(null);
 
         return false;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
index 3397772..eff893f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
@@ -32,8 +32,6 @@ import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.ACK_CLOSURE;
-
 /**
  * Grid client for NIO server.
  */
@@ -115,18 +113,9 @@ public class GridTcpNioCommunicationClient extends GridAbstractCommunicationClie
             // Node ID is never provided in asynchronous send mode.
             assert nodeId == null;
 
-            if (c != null)
-                ses.addMeta(ACK_CLOSURE.ordinal(), c);
-
-            ses.sendNoFuture(msg);
-
-            if (c != null)
-                ses.removeMeta(ACK_CLOSURE.ordinal());
+            ses.sendNoFuture(msg, c);
         }
         catch (IgniteCheckedException e) {
-            if (c != null)
-                ses.removeMeta(ACK_CLOSURE.ordinal());
-
             if (log.isDebugEnabled())
                 log.debug("Failed to send message [client=" + this + ", err=" + e + ']');
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java
index 508c791..e24f3ba 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java
@@ -42,13 +42,6 @@ public interface SessionWriteRequest {
     public boolean skipRecovery();
 
     /**
-     * Sets ack closure which will be applied when ack received.
-     *
-     * @param c Ack closure.
-     */
-    public void ackClosure(IgniteInClosure<IgniteException> c);
-
-    /**
      * The method will be called when ack received.
      */
     public void onAckReceived();

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java
index 8ed7db0..b4bd34a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java
@@ -34,6 +34,7 @@ import org.apache.ignite.internal.util.nio.GridNioFutureImpl;
 import org.apache.ignite.internal.util.nio.GridNioSession;
 import org.apache.ignite.internal.util.nio.GridNioSessionMetaKey;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
 
 import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.SSL_META;
 
@@ -285,10 +286,11 @@ public class GridNioSslFilter extends GridNioFilterAdapter {
     @Override public GridNioFuture<?> onSessionWrite(
         GridNioSession ses,
         Object msg,
-        boolean fut
+        boolean fut,
+        IgniteInClosure<IgniteException> ackC
     ) throws IgniteCheckedException {
         if (directMode)
-            return proceedSessionWrite(ses, msg, fut);
+            return proceedSessionWrite(ses, msg, fut, ackC);
 
         ByteBuffer input = checkMessage(ses, msg);
 
@@ -307,13 +309,13 @@ public class GridNioSslFilter extends GridNioFilterAdapter {
             if (hnd.isHandshakeFinished()) {
                 hnd.encrypt(input);
 
-                return hnd.writeNetBuffer();
+                return hnd.writeNetBuffer(ackC);
             }
             else {
                 if (log.isDebugEnabled())
                     log.debug("Write request received during handshake, scheduling deferred write: " + ses);
 
-                return hnd.deferredWrite(input);
+                return hnd.deferredWrite(input, ackC);
             }
         }
         catch (SSLException e) {
@@ -390,7 +392,7 @@ public class GridNioSslFilter extends GridNioFilterAdapter {
         try {
             hnd.closeOutbound();
 
-            hnd.writeNetBuffer();
+            hnd.writeNetBuffer(null);
         }
         catch (SSLException e) {
             U.warn(log, "Failed to shutdown SSL session gracefully (will force close) [ex=" + e + ", ses=" + ses + ']');

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java
index 269e8b9..e268716 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java
@@ -27,6 +27,7 @@ import javax.net.ssl.SSLEngineResult;
 import javax.net.ssl.SSLException;
 import javax.net.ssl.SSLSession;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.util.nio.GridNioEmbeddedFuture;
 import org.apache.ignite.internal.util.nio.GridNioException;
@@ -34,6 +35,7 @@ import org.apache.ignite.internal.util.nio.GridNioFuture;
 import org.apache.ignite.internal.util.nio.GridNioFutureImpl;
 import org.apache.ignite.internal.util.nio.GridNioSession;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
 
 import static javax.net.ssl.SSLEngineResult.HandshakeStatus;
 import static javax.net.ssl.SSLEngineResult.HandshakeStatus.FINISHED;
@@ -274,7 +276,7 @@ class GridNioSslHandler extends ReentrantLock {
                             log.debug("Wrapped handshake data [status=" + res.getStatus() + ", handshakeStatus=" +
                                 handshakeStatus + ", ses=" + ses + ']');
 
-                        writeNetBuffer();
+                        writeNetBuffer(null);
 
                         break;
                     }
@@ -412,16 +414,17 @@ class GridNioSslHandler extends ReentrantLock {
      * Adds write request to the queue.
      *
      * @param buf Buffer to write.
+     * @param ackC Closure invoked when message ACK is received.
      * @return Write future.
      */
-    GridNioFuture<?> deferredWrite(ByteBuffer buf) {
+    GridNioFuture<?> deferredWrite(ByteBuffer buf, IgniteInClosure<IgniteException> ackC) {
         assert isHeldByCurrentThread();
 
         GridNioEmbeddedFuture<Object> fut = new GridNioEmbeddedFuture<>();
 
         ByteBuffer cp = copy(buf);
 
-        deferredWriteQueue.offer(new WriteRequest(fut, cp));
+        deferredWriteQueue.offer(new WriteRequest(fut, cp, ackC));
 
         return fut;
     }
@@ -437,7 +440,7 @@ class GridNioSslHandler extends ReentrantLock {
         while (!deferredWriteQueue.isEmpty()) {
             WriteRequest req = deferredWriteQueue.poll();
 
-            req.future().onDone((GridNioFuture<Object>)parent.proceedSessionWrite(ses, req.buffer(), true));
+            req.future().onDone((GridNioFuture<Object>)parent.proceedSessionWrite(ses, req.buffer(), true, req.ackC));
         }
     }
 
@@ -475,14 +478,15 @@ class GridNioSslHandler extends ReentrantLock {
      * Copies data from out net buffer and passes it to the underlying chain.
      *
      * @return Write future.
+     * @param ackC Closure invoked when message ACK is received.
      * @throws GridNioException If send failed.
      */
-    GridNioFuture<?> writeNetBuffer() throws IgniteCheckedException {
+    GridNioFuture<?> writeNetBuffer(IgniteInClosure<IgniteException> ackC) throws IgniteCheckedException {
         assert isHeldByCurrentThread();
 
         ByteBuffer cp = copy(outNetBuf);
 
-        return parent.proceedSessionWrite(ses, cp, true);
+        return parent.proceedSessionWrite(ses, cp, true, ackC);
     }
 
     /**
@@ -670,20 +674,27 @@ class GridNioSslHandler extends ReentrantLock {
      */
     private static class WriteRequest {
         /** Future that should be completed. */
-        private GridNioEmbeddedFuture<Object> fut;
+        private final GridNioEmbeddedFuture<Object> fut;
 
         /** Buffer needed to be written. */
-        private ByteBuffer buf;
+        private final ByteBuffer buf;
+
+        /** */
+        private final IgniteInClosure<IgniteException> ackC;
 
         /**
          * Creates write request.
          *
          * @param fut Future.
          * @param buf Buffer to write.
+         * @param ackC Closure invoked when message ACK is received.
          */
-        private WriteRequest(GridNioEmbeddedFuture<Object> fut, ByteBuffer buf) {
+        private WriteRequest(GridNioEmbeddedFuture<Object> fut,
+            ByteBuffer buf,
+            IgniteInClosure<IgniteException> ackC) {
             this.fut = fut;
             this.buf = buf;
+            this.ackC = ackC;
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 35568c3..fe915e5 100755
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -369,7 +369,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         log.debug("Sending local node ID to newly accepted session: " + ses);
 
                     try {
-                        ses.sendNoFuture(nodeIdMessage());
+                        ses.sendNoFuture(nodeIdMessage(), null);
                     }
                     catch (IgniteCheckedException e) {
                         U.error(log, "Failed to send message: " + e, e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/thread/IgniteThread.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThread.java b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThread.java
index 8d88876..6005ac9 100644
--- a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThread.java
+++ b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThread.java
@@ -46,19 +46,19 @@ public class IgniteThread extends Thread {
     /** The name of the Ignite instance this thread belongs to. */
     protected final String igniteInstanceName;
 
-    /** Group index. */
-    private final int grpIdx;
-
     /** */
     private int compositeRwLockIdx;
 
+    /** */
+    private final int stripe;
+
     /**
      * Creates thread with given worker.
      *
      * @param worker Runnable to create thread with.
      */
     public IgniteThread(GridWorker worker) {
-        this(DFLT_GRP, worker.igniteInstanceName(), worker.name(), worker, GRP_IDX_UNASSIGNED);
+        this(DFLT_GRP, worker.igniteInstanceName(), worker.name(), worker, GRP_IDX_UNASSIGNED, -1);
     }
 
     /**
@@ -69,7 +69,7 @@ public class IgniteThread extends Thread {
      * @param r Runnable to execute.
      */
     public IgniteThread(String igniteInstanceName, String threadName, Runnable r) {
-        this(igniteInstanceName, threadName, r, GRP_IDX_UNASSIGNED);
+        this(igniteInstanceName, threadName, r, GRP_IDX_UNASSIGNED, -1);
     }
 
     /**
@@ -79,9 +79,10 @@ public class IgniteThread extends Thread {
      * @param threadName Name of thread.
      * @param r Runnable to execute.
      * @param grpIdx Index within a group.
+     * @param stripe Non-negative stripe number if this thread is striped pool thread.
      */
-    public IgniteThread(String igniteInstanceName, String threadName, Runnable r, int grpIdx) {
-        this(DFLT_GRP, igniteInstanceName, threadName, r, grpIdx);
+    public IgniteThread(String igniteInstanceName, String threadName, Runnable r, int grpIdx, int stripe) {
+        this(DFLT_GRP, igniteInstanceName, threadName, r, grpIdx, stripe);
     }
 
     /**
@@ -93,14 +94,16 @@ public class IgniteThread extends Thread {
      * @param threadName Name of thread.
      * @param r Runnable to execute.
      * @param grpIdx Thread index within a group.
+     * @param stripe Non-negative stripe number if this thread is striped pool thread.
      */
-    public IgniteThread(ThreadGroup grp, String igniteInstanceName, String threadName, Runnable r, int grpIdx) {
+    public IgniteThread(ThreadGroup grp, String igniteInstanceName, String threadName, Runnable r, int grpIdx, int stripe) {
         super(grp, r, createName(cntr.incrementAndGet(), threadName, igniteInstanceName));
 
         A.ensure(grpIdx >= -1, "grpIdx >= -1");
 
         this.igniteInstanceName = igniteInstanceName;
-        this.grpIdx = compositeRwLockIdx = grpIdx;
+        this.compositeRwLockIdx = grpIdx;
+        this.stripe = stripe;
     }
 
     /**
@@ -112,18 +115,15 @@ public class IgniteThread extends Thread {
         super(threadGrp, threadName);
 
         this.igniteInstanceName = igniteInstanceName;
-        this.grpIdx = compositeRwLockIdx = GRP_IDX_UNASSIGNED;
+        this.compositeRwLockIdx = GRP_IDX_UNASSIGNED;
+        this.stripe = -1;
     }
 
     /**
-     * Gets name of the grid this thread belongs to.
-     *
-     * @return Name of the grid this thread belongs to.
-     * @deprecated use {@link #getIgniteInstanceName()}
+     * @return Non-negative stripe number if this thread is striped pool thread.
      */
-    @Deprecated
-    public String getGridName() {
-        return getIgniteInstanceName();
+    public int stripe() {
+        return stripe;
     }
 
     /**
@@ -136,13 +136,6 @@ public class IgniteThread extends Thread {
     }
 
     /**
-     * @return Group index.
-     */
-    public int groupIndex() {
-        return grpIdx;
-    }
-
-    /**
      * @return Composite RW lock index.
      */
     public int compositeRwLockIndex() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java
index 119ef70..d2f0b15 100644
--- a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java
@@ -61,7 +61,7 @@ public class IgniteThreadFactory implements ThreadFactory {
 
     /** {@inheritDoc} */
     @Override public Thread newThread(@NotNull Runnable r) {
-        return new IgniteThread(igniteInstanceName, threadName, r, idxGen.incrementAndGet());
+        return new IgniteThread(igniteInstanceName, threadName, r, idxGen.incrementAndGet(), -1);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationSendMessageSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationSendMessageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationSendMessageSelfTest.java
index da2923f..8cbb596 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationSendMessageSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationSendMessageSelfTest.java
@@ -47,7 +47,7 @@ public class GridCommunicationSendMessageSelfTest extends GridCommonAbstractTest
     private static final int SAMPLE_CNT = 1;
 
     /** */
-    private static final byte DIRECT_TYPE = (byte)210;
+    private static final byte DIRECT_TYPE = -127;
 
     /** */
     private int bufSize;

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java
index 7c0e485..a158f7a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java
@@ -40,6 +40,13 @@ public class CacheRebalancingSelfTest extends GridCommonAbstractTest {
         return cfg;
     }
 
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        super.afterTest();
+    }
+
     /**
      * @throws Exception If failed.
      */
@@ -63,13 +70,12 @@ public class CacheRebalancingSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @param future Future.
+     * @param fut Future.
      * @return Internal future.
      */
-    private static IgniteInternalFuture internalFuture(IgniteFuture future) {
-        assert future instanceof IgniteFutureImpl;
+    private static IgniteInternalFuture internalFuture(IgniteFuture fut) {
+        assert fut instanceof IgniteFutureImpl : fut;
 
-        return ((IgniteFutureImpl)future).internalFuture();
+        return ((IgniteFutureImpl) fut).internalFuture();
     }
-
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
index 52c0ac5..09a0d9a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
@@ -78,8 +78,6 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
 
         cfg.setNetworkTimeout(60_000);
 
-        cfg.getTransactionConfiguration().setTxSerializableEnabled(true);
-
         TcpDiscoverySpi discoSpi = (TcpDiscoverySpi)cfg.getDiscoverySpi();
 
         discoSpi.setSocketTimeout(30_000);

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java
index 76cf78c..a12b6b9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java
@@ -94,13 +94,6 @@ public class GridCacheAtomicMessageCountSelfTest extends GridCommonAbstractTest
     /**
      * @throws Exception If failed.
      */
-    public void testPartitionedClock() throws Exception {
-        checkMessages(false, CLOCK);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     public void testPartitionedPrimary() throws Exception {
         checkMessages(false, PRIMARY);
     }
@@ -108,13 +101,6 @@ public class GridCacheAtomicMessageCountSelfTest extends GridCommonAbstractTest
     /**
      * @throws Exception If failed.
      */
-    public void testClientClock() throws Exception {
-        checkMessages(true, CLOCK);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     public void testClientPrimary() throws Exception {
         checkMessages(true, PRIMARY);
     }
@@ -206,14 +192,14 @@ public class GridCacheAtomicMessageCountSelfTest extends GridCommonAbstractTest
         private Map<Class<?>, AtomicInteger> cntMap = new HashMap<>();
 
         /** {@inheritDoc} */
-        @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure<IgniteException> ackClosure)
+        @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure<IgniteException> ackC)
             throws IgniteSpiException {
             AtomicInteger cntr = cntMap.get(((GridIoMessage)msg).message().getClass());
 
             if (cntr != null)
                 cntr.incrementAndGet();
 
-            super.sendMessage(node, msg, ackClosure);
+            super.sendMessage(node, msg, ackC);
         }
 
         /**
@@ -221,7 +207,7 @@ public class GridCacheAtomicMessageCountSelfTest extends GridCommonAbstractTest
          *
          * @param cls Class to count.
          */
-        public void registerMessage(Class<?> cls) {
+        void registerMessage(Class<?> cls) {
             AtomicInteger cntr = cntMap.get(cls);
 
             if (cntr == null)
@@ -232,7 +218,7 @@ public class GridCacheAtomicMessageCountSelfTest extends GridCommonAbstractTest
          * @param cls Message type to get count.
          * @return Number of messages of given class.
          */
-        public int messageCount(Class<?> cls) {
+        int messageCount(Class<?> cls) {
             AtomicInteger cntr = cntMap.get(cls);
 
             return cntr == null ? 0 : cntr.get();

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
index ba37974..dd27d72 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
@@ -66,6 +66,7 @@ import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.eventstorage.memory.MemoryEventStorageSpi;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.jetbrains.annotations.Nullable;

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
index 728bf13..a44e49e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
@@ -401,9 +401,6 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
 
         assertEquals(3, msgs.size());
 
-        for (Object msg : msgs)
-            assertTrue(((GridNearAtomicFullUpdateRequest)msg).clientRequest());
-
         map.put(primaryKey(ignite0.cache(null)), 3);
         map.put(primaryKey(ignite1.cache(null)), 4);
         map.put(primaryKey(ignite2.cache(null)), 5);
@@ -1693,8 +1690,6 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
      * @throws Exception If failed.
      */
     public void testAtomicPrimaryPutAllMultinode() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1685");
-
         multinode(PRIMARY, TestType.PUT_ALL);
     }
 
@@ -1702,8 +1697,6 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
      * @throws Exception If failed.
      */
     public void testAtomicClockPutAllMultinode() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1685");
-
         multinode(CLOCK, TestType.PUT_ALL);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java
index cb1f6fb4..1d2cd2d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java
@@ -103,7 +103,7 @@ public abstract class IgniteCacheMessageRecoveryAbstractTest extends GridCommonA
         for (int i = 0; i < GRID_CNT; i++) {
             final IgniteKernal grid = (IgniteKernal)grid(i);
 
-            GridTestUtils.retryAssert(log, 10, 100, new CA() {
+            GridTestUtils.retryAssert(log, 10, 500, new CA() {
                 @Override public void apply() {
                     assertTrue(grid.internalCache().context().mvcc().atomicFutures().isEmpty());
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAtomicNearCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAtomicNearCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAtomicNearCacheSelfTest.java
index 5050300..2600e7b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAtomicNearCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAtomicNearCacheSelfTest.java
@@ -39,9 +39,11 @@ import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
@@ -702,8 +704,12 @@ public class GridCacheAtomicNearCacheSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     @SuppressWarnings("ConstantConditions")
-    private void checkEntry(Ignite ignite, Integer key, @Nullable Integer val, boolean expectNear, UUID... expReaders)
-        throws Exception {
+    private void checkEntry(Ignite ignite,
+        Integer key,
+        @Nullable Integer val,
+        boolean expectNear,
+        final UUID... expReaders) throws Exception
+    {
         GridCacheAdapter<Integer, Integer> near = ((IgniteKernal) ignite).internalCache();
 
         assertTrue(near.isNear());
@@ -728,11 +734,22 @@ public class GridCacheAtomicNearCacheSelfTest extends GridCommonAbstractTest {
 
         GridDhtCacheAdapter<Integer, Integer> dht = ((GridNearCacheAdapter<Integer, Integer>)near).dht();
 
-        GridDhtCacheEntry dhtEntry = (GridDhtCacheEntry)dht.peekEx(key);
+        final GridDhtCacheEntry dhtEntry = (GridDhtCacheEntry)dht.peekEx(key);
 
         if (expectDht) {
             assertNotNull("No dht entry for: " + key + ", grid: " + ignite.name(), dhtEntry);
 
+            GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                @Override public boolean apply() {
+                    try {
+                        return dhtEntry.readers().size() == expReaders.length;
+                    }
+                    catch (Exception e) {
+                        throw new RuntimeException(e);
+                    }
+                }
+            }, 5000);
+
             Collection<UUID> readers = dhtEntry.readers();
 
             assertEquals(expReaders.length, readers.size());

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
index 3942e35..2971f81 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
@@ -43,10 +43,10 @@ import org.apache.ignite.configuration.AtomicConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
@@ -152,7 +152,12 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCommonAbst
     @Override protected void afterTest() throws Exception {
         super.afterTest();
 
-        ignite(0).destroyCache(null);
+        try {
+            checkInternalCleanup();
+        }
+        finally {
+            ignite(0).destroyCache(null);
+        }
     }
 
     /**
@@ -456,22 +461,40 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCommonAbst
 
         for (int i = 0; i < keysCnt; i++)
             assertEquals((Integer)iter, cache.get(i));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkInternalCleanup() throws Exception{
+        checkNoAtomicFutures();
+
+        checkOnePhaseCommitReturnValuesCleaned();
+    }
 
+    /**
+     * @throws Exception If failed.
+     */
+    void checkNoAtomicFutures() throws Exception {
         for (int i = 0; i < GRID_CNT; i++) {
-            IgniteKernal ignite = (IgniteKernal)grid(i);
+            final IgniteKernal ignite = (IgniteKernal)grid(i);
+
+            GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                @Override public boolean apply() {
+                    return ignite.context().cache().context().mvcc().atomicFuturesCount() == 0;
+                }
+            }, 5_000);
 
             Collection<?> futs = ignite.context().cache().context().mvcc().atomicFutures();
 
             assertTrue("Unexpected atomic futures: " + futs, futs.isEmpty());
         }
-
-        checkOnePhaseCommitReturnValuesCleaned();
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
-    protected void checkOnePhaseCommitReturnValuesCleaned() throws IgniteInterruptedCheckedException {
+    void checkOnePhaseCommitReturnValuesCleaned() throws Exception {
         U.sleep(DEFERRED_ONE_PHASE_COMMIT_ACK_REQUEST_TIMEOUT);
 
         for (int i = 0; i < GRID_CNT; i++) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
index cc5f548..7460828 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
@@ -208,7 +208,7 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testOriginatingNodeFailureForcesOnePhaseCommitDataCleanup() throws Exception {
         ignite(0).createCache(cacheConfiguration(TestMemoryMode.HEAP, false));


[3/8] ignite git commit: ignite-4705 Atomic cache protocol change: notify client node from backups

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
index f182ecb..a44ccf9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
@@ -36,7 +36,6 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheEntryPredicate;
 import org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException;
 import org.apache.ignite.internal.processors.cache.EntryProcessorResourceInjectorProxy;
-import org.apache.ignite.internal.processors.cache.GridCacheAffinityManager;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheOperation;
 import org.apache.ignite.internal.processors.cache.GridCacheReturn;
@@ -47,10 +46,8 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearAtom
 import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
-import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.CI1;
-import org.apache.ignite.internal.util.typedef.CI2;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.CU;
@@ -58,18 +55,14 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM;
 
 /**
  * DHT atomic cache near update future.
  */
 public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFuture {
-    /** Fast map flag. */
-    private final boolean fastMap;
-
     /** Keys */
     private Collection<?> keys;
 
@@ -87,13 +80,18 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
 
     /** Mappings if operations is mapped to more than one node. */
     @GridToStringInclude
-    private Map<UUID, GridNearAtomicFullUpdateRequest> mappings;
+    private Map<UUID, PrimaryRequestState> mappings;
 
     /** Keys to remap. */
+    @GridToStringInclude
     private Collection<KeyCacheObject> remapKeys;
 
     /** Not null is operation is mapped to single node. */
-    private GridNearAtomicFullUpdateRequest singleReq;
+    @GridToStringInclude
+    private PrimaryRequestState singleReq;
+
+    /** */
+    private int resCnt;
 
     /**
      * @param cctx Cache context.
@@ -149,84 +147,124 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
         this.vals = vals;
         this.conflictPutVals = conflictPutVals;
         this.conflictRmvVals = conflictRmvVals;
-
-        fastMap = cache.isFastMap(filter, op);
     }
 
     /** {@inheritDoc} */
-    @Override public GridCacheVersion version() {
+    @Override public Long id() {
         synchronized (mux) {
-            return futVer;
+            return futId;
         }
     }
 
     /** {@inheritDoc} */
     @Override public boolean onNodeLeft(UUID nodeId) {
-        GridNearAtomicUpdateResponse res = null;
+        GridCacheReturn opRes0 = null;
+        CachePartialUpdateCheckedException err0 = null;
+        AffinityTopologyVersion remapTopVer0 = null;
+
+        boolean rcvAll = false;
 
-        GridNearAtomicFullUpdateRequest req;
+        List<GridNearAtomicCheckUpdateRequest> checkReqs = null;
 
         synchronized (mux) {
-            if (singleReq != null)
-                req = singleReq.nodeId().equals(nodeId) ? singleReq : null;
-            else
-                req = mappings != null ? mappings.get(nodeId) : null;
+            if (futId == null)
+                return false;
 
-            if (req != null && req.response() == null) {
-                res = new GridNearAtomicUpdateResponse(cctx.cacheId(),
-                    nodeId,
-                    req.futureVersion(),
-                    cctx.deploymentEnabled());
+            if (singleReq != null) {
+                if (singleReq.req.nodeId.equals(nodeId)) {
+                    GridNearAtomicAbstractUpdateRequest req = singleReq.onPrimaryFail();
 
-                ClusterTopologyCheckedException e = new ClusterTopologyCheckedException("Primary node left grid " +
-                    "before response is received: " + nodeId);
+                    if (req != null) {
+                        rcvAll = true;
 
-                e.retryReadyFuture(cctx.shared().nextAffinityReadyFuture(req.topologyVersion()));
+                        GridNearAtomicUpdateResponse res = primaryFailedResponse(req);
 
-                res.addFailedKeys(req.keys(), e);
-            }
-        }
+                        singleReq.onPrimaryResponse(res, cctx);
+
+                        onPrimaryError(req, res);
+                    }
+                }
+                else {
+                    DhtLeftResult res = singleReq.onDhtNodeLeft(nodeId);
+
+                    if (res == DhtLeftResult.DONE)
+                        rcvAll = true;
+                    else if (res == DhtLeftResult.ALL_RCVD_CHECK_PRIMARY)
+                        checkReqs = Collections.singletonList(new GridNearAtomicCheckUpdateRequest(singleReq.req));
+                }
 
-        if (res != null) {
-            if (msgLog.isDebugEnabled()) {
-                msgLog.debug("Near update fut, node left [futId=" + req.futureVersion() +
-                    ", writeVer=" + req.updateVersion() +
-                    ", node=" + nodeId + ']');
+                if (rcvAll) {
+                    opRes0 = opRes;
+                    err0 = err;
+                    remapTopVer0 = onAllReceived();
+                }
             }
+            else {
+                if (mappings == null)
+                    return false;
 
-            onResult(nodeId, res, true);
-        }
+                for (Map.Entry<UUID, PrimaryRequestState> e : mappings.entrySet()) {
+                    assert e.getKey().equals(e.getValue().req.nodeId());
 
-        return false;
-    }
+                    PrimaryRequestState reqState = e.getValue();
 
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Void> completeFuture(AffinityTopologyVersion topVer) {
-        // Wait fast-map near atomic update futures in CLOCK mode.
-        if (fastMap) {
-            GridFutureAdapter<Void> fut;
+                    boolean reqDone = false;
 
-            synchronized (mux) {
-                if (this.topVer != AffinityTopologyVersion.ZERO && this.topVer.compareTo(topVer) < 0) {
-                    if (topCompleteFut == null)
-                        topCompleteFut = new GridFutureAdapter<>();
+                    if (e.getKey().equals(nodeId)) {
+                        GridNearAtomicAbstractUpdateRequest req = reqState.onPrimaryFail();
 
-                    fut = topCompleteFut;
-                }
-                else
-                    fut = null;
-            }
+                        if (req != null) {
+                            reqDone = true;
+
+                            GridNearAtomicUpdateResponse res = primaryFailedResponse(req);
+
+                            reqState.onPrimaryResponse(res, cctx);
+
+                            onPrimaryError(req, res);
+                        }
+                    }
+                    else {
+                        DhtLeftResult res = reqState.onDhtNodeLeft(nodeId);
+
+                        if (res == DhtLeftResult.DONE)
+                            reqDone = true;
+                        else if (res == DhtLeftResult.ALL_RCVD_CHECK_PRIMARY) {
+                            if (checkReqs == null)
+                                checkReqs = new ArrayList<>();
 
-            if (fut != null && isDone()) {
-                fut.onDone();
+                            checkReqs.add(new GridNearAtomicCheckUpdateRequest(reqState.req));
+                        }
+                    }
+
+                    if (reqDone) {
+                        assert mappings.size() > resCnt : "[mappings=" + mappings.size() + ", cnt=" + resCnt + ']';
+
+                        resCnt++;
+
+                        if (mappings.size() == resCnt) {
+                            rcvAll = true;
 
-                return null;
+                            opRes0 = opRes;
+                            err0 = err;
+                            remapTopVer0 = onAllReceived();
+
+                            break;
+                        }
+                    }
+                }
             }
+        }
 
-            return fut;
+        if (checkReqs != null) {
+            assert !rcvAll;
+
+            for (int i = 0; i < checkReqs.size(); i++)
+                sendCheckUpdateRequest(checkReqs.get(i));
         }
+        else if (rcvAll)
+            finishUpdateFuture(opRes0, err0, remapTopVer0);
 
-        return null;
+        return false;
     }
 
     /** {@inheritDoc} */
@@ -244,10 +282,10 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
             retval = Collections.emptyMap();
 
         if (super.onDone(retval, err)) {
-            GridCacheVersion futVer = onFutureDone();
+            Long futId = onFutureDone();
 
-            if (futVer != null)
-                cctx.mvcc().removeAtomicFuture(futVer);
+            if (futId != null)
+                cctx.mvcc().removeAtomicFuture(futId);
 
             return true;
         }
@@ -256,145 +294,166 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings({"unchecked", "ThrowableResultOfMethodCallIgnored"})
-    @Override public void onResult(UUID nodeId, GridNearAtomicUpdateResponse res, boolean nodeErr) {
-        GridNearAtomicFullUpdateRequest req;
-
-        AffinityTopologyVersion remapTopVer = null;
-
-        GridCacheReturn opRes0 = null;
-        CachePartialUpdateCheckedException err0 = null;
-
-        boolean rcvAll;
-
-        GridFutureAdapter<?> fut0 = null;
+    @Override public void onDhtResponse(UUID nodeId, GridDhtAtomicNearResponse res) {
+        GridCacheReturn opRes0;
+        CachePartialUpdateCheckedException err0;
+        AffinityTopologyVersion remapTopVer0;
 
         synchronized (mux) {
-            if (!res.futureVersion().equals(futVer))
+            if (futId == null || futId != res.futureId())
                 return;
 
-            if (singleReq != null) {
-                if (!singleReq.nodeId().equals(nodeId))
-                    return;
+            PrimaryRequestState reqState;
 
-                req = singleReq;
+            if (singleReq != null) {
+                assert singleReq.req.nodeId().equals(res.primaryId());
 
-                singleReq = null;
+                if (opRes == null && res.hasResult())
+                    opRes = res.result();
 
-                rcvAll = true;
+                if (singleReq.onDhtResponse(nodeId, res)) {
+                    opRes0 = opRes;
+                    err0 = err;
+                    remapTopVer0 = onAllReceived();
+                }
+                else
+                    return;
             }
             else {
-                req = mappings != null ? mappings.get(nodeId) : null;
+                reqState = mappings != null ? mappings.get(res.primaryId()) : null;
 
-                if (req != null && req.onResponse(res)) {
-                    resCnt++;
+                if (reqState != null) {
+                    if (opRes == null && res.hasResult())
+                        opRes = res.result();
 
-                    rcvAll = mappings.size() == resCnt;
+                    if (reqState.onDhtResponse(nodeId, res)) {
+                        assert mappings.size() > resCnt : "[mappings=" + mappings.size() + ", cnt=" + resCnt + ']';
+
+                        resCnt++;
+
+                        if (mappings.size() == resCnt) {
+                            opRes0 = opRes;
+                            err0 = err;
+                            remapTopVer0 = onAllReceived();
+                        }
+                        else
+                            return;
+                    }
+                    else
+                        return;
                 }
                 else
                     return;
             }
+        }
 
-            assert req != null && req.topologyVersion().equals(topVer) : req;
+        UpdateErrors errors = res.errors();
 
-            if (res.remapKeys() != null) {
-                assert !fastMap || cctx.kernalContext().clientNode();
+        if (errors != null) {
+            assert errors.error() != null;
 
-                if (remapKeys == null)
-                    remapKeys = U.newHashSet(res.remapKeys().size());
+            onDone(errors.error());
+
+            return;
+        }
 
-                remapKeys.addAll(res.remapKeys());
+        finishUpdateFuture(opRes0, err0, remapTopVer0);
+    }
 
-                if (mapErrTopVer == null || mapErrTopVer.compareTo(req.topologyVersion()) < 0)
-                    mapErrTopVer = req.topologyVersion();
-            }
-            else if (res.error() != null) {
-                if (res.failedKeys() != null) {
-                    if (err == null)
-                        err = new CachePartialUpdateCheckedException(
-                            "Failed to update keys (retry update if possible).");
+    /** {@inheritDoc} */
+    @SuppressWarnings({"unchecked", "ThrowableResultOfMethodCallIgnored"})
+    @Override public void onPrimaryResponse(UUID nodeId, GridNearAtomicUpdateResponse res, boolean nodeErr) {
+        GridNearAtomicAbstractUpdateRequest req;
 
-                    Collection<Object> keys = new ArrayList<>(res.failedKeys().size());
+        AffinityTopologyVersion remapTopVer0 = null;
 
-                    for (KeyCacheObject key : res.failedKeys())
-                        keys.add(cctx.cacheObjectContext().unwrapBinaryIfNeeded(key, keepBinary, false));
+        GridCacheReturn opRes0 = null;
+        CachePartialUpdateCheckedException err0 = null;
 
-                    err.add(keys, res.error(), req.topologyVersion());
-                }
+        boolean rcvAll;
+
+        synchronized (mux) {
+            if (futId == null || futId != res.futureId())
+                return;
+
+            if (singleReq != null) {
+                req = singleReq.processPrimaryResponse(nodeId, res);
+
+                if (req == null)
+                    return;
+
+                rcvAll = singleReq.onPrimaryResponse(res, cctx);
             }
             else {
-                if (!req.fastMap() || req.hasPrimary()) {
-                    GridCacheReturn ret = res.returnValue();
-
-                    if (op == TRANSFORM) {
-                        if (ret != null) {
-                            assert ret.value() == null || ret.value() instanceof Map : ret.value();
-
-                            if (ret.value() != null) {
-                                if (opRes != null)
-                                    opRes.mergeEntryProcessResults(ret);
-                                else
-                                    opRes = ret;
-                            }
-                        }
-                    }
-                    else
-                        opRes = ret;
-                }
-            }
+                if (mappings == null)
+                    return;
 
-            if (rcvAll) {
-                if (remapKeys != null) {
-                    assert mapErrTopVer != null;
+                PrimaryRequestState reqState = mappings.get(nodeId);
 
-                    remapTopVer = cctx.shared().exchange().topologyVersion();
-                }
-                else {
-                    if (err != null &&
-                        X.hasCause(err, CachePartialUpdateCheckedException.class) &&
-                        X.hasCause(err, ClusterTopologyCheckedException.class) &&
-                        storeFuture() &&
-                        --remapCnt > 0) {
-                        ClusterTopologyCheckedException topErr =
-                            X.cause(err, ClusterTopologyCheckedException.class);
+                if (reqState == null)
+                    return;
 
-                        if (!(topErr instanceof ClusterTopologyServerNotFoundException)) {
-                            CachePartialUpdateCheckedException cause =
-                                X.cause(err, CachePartialUpdateCheckedException.class);
+                req = reqState.processPrimaryResponse(nodeId, res);
+
+                if (req != null) {
+                    if (reqState.onPrimaryResponse(res, cctx)) {
+                        assert mappings.size() > resCnt : "[mappings=" + mappings.size() + ", cnt=" + resCnt + ']';
+
+                        resCnt++;
+
+                        rcvAll = mappings.size() == resCnt;
+                    }
+                    else {
+                        assert mappings.size() > resCnt : "[mappings=" + mappings.size() + ", cnt=" + resCnt + ']';
+
+                        rcvAll = false;
+                    }
+                }
+                else
+                    return;
+            }
 
-                            assert cause != null && cause.topologyVersion() != null : err;
+            assert req.topologyVersion().equals(topVer) : req;
 
-                            remapTopVer =
-                                new AffinityTopologyVersion(cause.topologyVersion().topologyVersion() + 1);
+            if (res.remapTopologyVersion() != null) {
+                assert !req.topologyVersion().equals(res.remapTopologyVersion());
 
-                            err = null;
+                if (remapKeys == null)
+                    remapKeys = U.newHashSet(req.size());
 
-                            Collection<Object> failedKeys = cause.failedKeys();
+                remapKeys.addAll(req.keys());
 
-                            remapKeys = new ArrayList<>(failedKeys.size());
+                if (remapTopVer == null || remapTopVer.compareTo(res.remapTopologyVersion()) < 0)
+                    remapTopVer = req.topologyVersion();
+            }
+            else if (res.error() != null)
+                onPrimaryError(req, res);
+            else {
+                GridCacheReturn ret = res.returnValue();
 
-                            for (Object key : failedKeys)
-                                remapKeys.add(cctx.toCacheKeyObject(key));
+                if (op == TRANSFORM) {
+                    if (ret != null) {
+                        assert ret.value() == null || ret.value() instanceof Map : ret.value();
 
-                            updVer = null;
+                        if (ret.value() != null) {
+                            if (opRes != null)
+                                opRes.mergeEntryProcessResults(ret);
+                            else
+                                opRes = ret;
                         }
                     }
                 }
+                else
+                    opRes = ret;
+            }
+
+            if (rcvAll) {
+                remapTopVer0 = onAllReceived();
 
-                if (remapTopVer == null) {
+                if (remapTopVer0 == null) {
                     err0 = err;
                     opRes0 = opRes;
                 }
-                else {
-                    fut0 = topCompleteFut;
-
-                    topCompleteFut = null;
-
-                    cctx.mvcc().removeAtomicFuture(futVer);
-
-                    futVer = null;
-                    topVer = AffinityTopologyVersion.ZERO;
-                }
             }
         }
 
@@ -406,67 +465,160 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
 
         if (rcvAll && nearEnabled) {
             if (mappings != null) {
-                for (GridNearAtomicFullUpdateRequest req0 : mappings.values()) {
-                    GridNearAtomicUpdateResponse res0 = req0.response();
+                for (PrimaryRequestState reqState : mappings.values()) {
+                    GridNearAtomicUpdateResponse res0 = reqState.req.response();
 
-                    assert res0 != null : req0;
+                    assert res0 != null : reqState;
 
-                    updateNear(req0, res0);
+                    updateNear(reqState.req, res0);
                 }
             }
             else if (!nodeErr)
                 updateNear(req, res);
         }
 
-        if (remapTopVer != null) {
-            if (fut0 != null)
-                fut0.onDone();
+        if (remapTopVer0 != null) {
+            waitAndRemap(remapTopVer0);
 
-            if (!waitTopFut) {
-                onDone(new GridCacheTryPutFailedException());
+            return;
+        }
 
-                return;
+        if (rcvAll)
+            onDone(opRes0, err0);
+    }
+
+    private void waitAndRemap(AffinityTopologyVersion remapTopVer) {
+        assert remapTopVer != null;
+
+        if (!waitTopFut) {
+            onDone(new GridCacheTryPutFailedException());
+
+            return;
+        }
+
+        if (topLocked) {
+            assert !F.isEmpty(remapKeys) : remapKeys;
+
+            CachePartialUpdateCheckedException e =
+                new CachePartialUpdateCheckedException("Failed to update keys (retry update if possible).");
+
+            ClusterTopologyCheckedException cause = new ClusterTopologyCheckedException(
+                "Failed to update keys, topology changed while execute atomic update inside transaction.");
+
+            cause.retryReadyFuture(cctx.affinity().affinityReadyFuture(remapTopVer));
+
+            e.add(remapKeys, cause);
+
+            onDone(e);
+
+            return;
+        }
+
+        IgniteInternalFuture<AffinityTopologyVersion> fut = cctx.shared().exchange().affinityReadyFuture(remapTopVer);
+
+        if (fut == null)
+            fut = new GridFinishedFuture<>(remapTopVer);
+
+        fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
+            @Override public void apply(final IgniteInternalFuture<AffinityTopologyVersion> fut) {
+                cctx.kernalContext().closure().runLocalSafe(new Runnable() {
+                    @Override public void run() {
+                        mapOnTopology();
+                    }
+                });
             }
+        });
+    }
+
+    /**
+     * @return Non null topology version if update should be remapped.
+     */
+    @Nullable private AffinityTopologyVersion onAllReceived() {
+        assert futId != null;
 
-            if (topLocked) {
-                assert !F.isEmpty(remapKeys) : remapKeys;
+        AffinityTopologyVersion remapTopVer0 = null;
 
-                CachePartialUpdateCheckedException e =
-                    new CachePartialUpdateCheckedException("Failed to update keys (retry update if possible).");
+        if (remapKeys != null) {
+            assert remapTopVer != null;
 
-                ClusterTopologyCheckedException cause = new ClusterTopologyCheckedException(
-                    "Failed to update keys, topology changed while execute atomic update inside transaction.");
+            remapTopVer0 = remapTopVer;
+        }
+        else {
+            if (err != null &&
+                X.hasCause(err, CachePartialUpdateCheckedException.class) &&
+                X.hasCause(err, ClusterTopologyCheckedException.class) &&
+                storeFuture() &&
+                --remapCnt > 0) {
+                ClusterTopologyCheckedException topErr = X.cause(err, ClusterTopologyCheckedException.class);
 
-                cause.retryReadyFuture(cctx.affinity().affinityReadyFuture(remapTopVer));
+                if (!(topErr instanceof ClusterTopologyServerNotFoundException)) {
+                    CachePartialUpdateCheckedException cause =
+                        X.cause(err, CachePartialUpdateCheckedException.class);
 
-                e.add(remapKeys, cause);
+                    assert cause != null && cause.topologyVersion() != null : err;
+                    assert remapKeys == null;
+                    assert remapTopVer == null;
 
-                onDone(e);
+                    remapTopVer = remapTopVer0 =
+                        new AffinityTopologyVersion(cause.topologyVersion().topologyVersion() + 1);
 
-                return;
+                    err = null;
+
+                    Collection<Object> failedKeys = cause.failedKeys();
+
+                    remapKeys = new ArrayList<>(failedKeys.size());
+
+                    for (Object key : failedKeys)
+                        remapKeys.add(cctx.toCacheKeyObject(key));
+                }
             }
+        }
 
-            IgniteInternalFuture<AffinityTopologyVersion> fut =
-                cctx.shared().exchange().affinityReadyFuture(remapTopVer);
+        if (remapTopVer0 != null) {
+            cctx.mvcc().removeAtomicFuture(futId);
 
-            if (fut == null)
-                fut = new GridFinishedFuture<>(remapTopVer);
+            futId = null;
+            topVer = AffinityTopologyVersion.ZERO;
 
-            fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
-                @Override public void apply(final IgniteInternalFuture<AffinityTopologyVersion> fut) {
-                    cctx.kernalContext().closure().runLocalSafe(new Runnable() {
-                        @Override public void run() {
-                            mapOnTopology();
-                        }
-                    });
+            remapTopVer = null;
+        }
+
+        return remapTopVer0;
+    }
+
+    /**
+     * @param opRes Operation result.
+     * @param err Operation error.
+     */
+    private void finishUpdateFuture(GridCacheReturn opRes,
+        CachePartialUpdateCheckedException err,
+        @Nullable AffinityTopologyVersion remapTopVer) {
+        if (nearEnabled) {
+            if (mappings != null) {
+                for (PrimaryRequestState reqState : mappings.values()) {
+                    GridNearAtomicUpdateResponse res0 = reqState.req.response();
+
+                    assert res0 != null : reqState;
+
+                    updateNear(reqState.req, res0);
                 }
-            });
+            }
+            else {
+                assert singleReq != null && singleReq.req.response() != null;
+
+                updateNear(singleReq.req, singleReq.req.response());
+            }
+        }
+
+        if (remapTopVer != null) {
+            assert !F.isEmpty(remapKeys);
+
+            waitAndRemap(remapTopVer);
 
             return;
         }
 
-        if (rcvAll)
-            onDone(opRes0, err0);
+        onDone(opRes, err);
     }
 
     /**
@@ -475,10 +627,10 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
      * @param req Update request.
      * @param res Update response.
      */
-    private void updateNear(GridNearAtomicFullUpdateRequest req, GridNearAtomicUpdateResponse res) {
+    private void updateNear(GridNearAtomicAbstractUpdateRequest req, GridNearAtomicUpdateResponse res) {
         assert nearEnabled;
 
-        if (res.remapKeys() != null || !req.hasPrimary())
+        if (res.remapTopologyVersion() != null)
             return;
 
         GridNearAtomicCache near = (GridNearAtomicCache)cctx.dht().near();
@@ -489,59 +641,48 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
     /** {@inheritDoc} */
     @Override protected void mapOnTopology() {
         AffinityTopologyVersion topVer;
-        GridCacheVersion futVer;
-
-        cache.topology().readLock();
-
-        try {
-            if (cache.topology().stopping()) {
-                onDone(new IgniteCheckedException("Failed to perform cache operation (cache is stopped): " +
-                    cache.name()));
 
-                return;
-            }
+        if (cache.topology().stopping()) {
+            onDone(new IgniteCheckedException("Failed to perform cache operation (cache is stopped): " +
+                cache.name()));
 
-            GridDhtTopologyFuture fut = cache.topology().topologyVersionFuture();
-
-            if (fut.isDone()) {
-                Throwable err = fut.validateCache(cctx);
+            return;
+        }
 
-                if (err != null) {
-                    onDone(err);
+        GridDhtTopologyFuture fut = cache.topology().topologyVersionFuture();
 
-                    return;
-                }
+        if (fut.isDone()) {
+            Throwable err = fut.validateCache(cctx);
 
-                topVer = fut.topologyVersion();
-
-                futVer = addAtomicFuture(topVer);
-            }
-            else {
-                if (waitTopFut) {
-                    assert !topLocked : this;
-
-                    fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
-                        @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
-                            cctx.kernalContext().closure().runLocalSafe(new Runnable() {
-                                @Override public void run() {
-                                    mapOnTopology();
-                                }
-                            });
-                        }
-                    });
-                }
-                else
-                    onDone(new GridCacheTryPutFailedException());
+            if (err != null) {
+                onDone(err);
 
                 return;
             }
+
+            topVer = fut.topologyVersion();
         }
-        finally {
-            cache.topology().readUnlock();
+        else {
+            if (waitTopFut) {
+                assert !topLocked : this;
+
+                fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
+                    @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
+                        cctx.kernalContext().closure().runLocalSafe(new Runnable() {
+                            @Override public void run() {
+                                mapOnTopology();
+                            }
+                        });
+                    }
+                });
+            }
+            else
+                onDone(new GridCacheTryPutFailedException());
+
+            return;
         }
 
-        if (futVer != null)
-            map(topVer, futVer, remapKeys);
+        map(topVer, remapKeys);
     }
 
     /**
@@ -549,13 +690,15 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
      *
      * @param mappings Mappings to send.
      */
-    private void doUpdate(Map<UUID, GridNearAtomicFullUpdateRequest> mappings) {
+    private void sendUpdateRequests(Map<UUID, PrimaryRequestState> mappings) {
         UUID locNodeId = cctx.localNodeId();
 
-        GridNearAtomicFullUpdateRequest locUpdate = null;
+        GridNearAtomicAbstractUpdateRequest locUpdate = null;
 
         // Send messages to remote nodes first, then run local update.
-        for (GridNearAtomicFullUpdateRequest req : mappings.values()) {
+        for (PrimaryRequestState reqState : mappings.values()) {
+            GridNearAtomicAbstractUpdateRequest req = reqState.req;
+
             if (locNodeId.equals(req.nodeId())) {
                 assert locUpdate == null : "Cannot have more than one local mapping [locUpdate=" + locUpdate +
                     ", req=" + req + ']';
@@ -564,18 +707,22 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
             }
             else {
                 try {
+                    if (req.initMappingLocally() && reqState.dhtNodes.isEmpty()) {
+                        reqState.dhtNodes = null;
+
+                        req.needPrimaryResponse(true);
+                    }
+
                     cctx.io().send(req.nodeId(), req, cctx.ioPolicy());
 
                     if (msgLog.isDebugEnabled()) {
-                        msgLog.debug("Near update fut, sent request [futId=" + req.futureVersion() +
-                            ", writeVer=" + req.updateVersion() +
+                        msgLog.debug("Near update fut, sent request [futId=" + req.futureId() +
                             ", node=" + req.nodeId() + ']');
                     }
                 }
                 catch (IgniteCheckedException e) {
                     if (msgLog.isDebugEnabled()) {
-                        msgLog.debug("Near update fut, failed to send request [futId=" + req.futureVersion() +
-                            ", writeVer=" + req.updateVersion() +
+                        msgLog.debug("Near update fut, failed to send request [futId=" + req.futureId() +
                             ", node=" + req.nodeId() +
                             ", err=" + e + ']');
                     }
@@ -587,9 +734,12 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
 
         if (locUpdate != null) {
             cache.updateAllAsyncInternal(cctx.localNodeId(), locUpdate,
-                new CI2<GridNearAtomicFullUpdateRequest, GridNearAtomicUpdateResponse>() {
-                    @Override public void apply(GridNearAtomicFullUpdateRequest req, GridNearAtomicUpdateResponse res) {
-                        onResult(res.nodeId(), res, false);
+                new GridDhtAtomicCache.UpdateReplyClosure() {
+                    @Override public void apply(GridNearAtomicAbstractUpdateRequest req, GridNearAtomicUpdateResponse res) {
+                        if (syncMode != FULL_ASYNC)
+                            onPrimaryResponse(res.nodeId(), res, false);
+                        else if (res.remapTopologyVersion() != null)
+                            ((GridDhtAtomicCache)cctx.cache()).remapToNewPrimary(req);
                     }
                 });
         }
@@ -599,18 +749,15 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
     }
 
     /** {@inheritDoc} */
-    @Override protected void map(AffinityTopologyVersion topVer, GridCacheVersion futVer) {
-        map(topVer, futVer, null);
+    @Override protected void map(AffinityTopologyVersion topVer) {
+        map(topVer, null);
     }
 
     /**
      * @param topVer Topology version.
-     * @param futVer Future ID.
      * @param remapKeys Keys to remap.
      */
-    void map(AffinityTopologyVersion topVer,
-        GridCacheVersion futVer,
-        @Nullable Collection<KeyCacheObject> remapKeys) {
+    void map(AffinityTopologyVersion topVer, @Nullable Collection<KeyCacheObject> remapKeys) {
         Collection<ClusterNode> topNodes = CU.affinityNodes(cctx, topVer);
 
         if (F.isEmpty(topNodes)) {
@@ -620,64 +767,45 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
             return;
         }
 
-        GridCacheVersion updVer;
-
-        // Assign version on near node in CLOCK ordering mode even if fastMap is false.
-        if (cctx.config().getAtomicWriteOrderMode() == CLOCK) {
-            updVer = this.updVer;
-
-            if (updVer == null) {
-                updVer = futVer;
-
-                if (log.isDebugEnabled())
-                    log.debug("Assigned fast-map version for update on near node: " + updVer);
-            }
-        }
-        else
-            updVer = null;
+        Long futId = cctx.mvcc().atomicFutureId();
 
         Exception err = null;
-        GridNearAtomicFullUpdateRequest singleReq0 = null;
-        Map<UUID, GridNearAtomicFullUpdateRequest> mappings0 = null;
+        PrimaryRequestState singleReq0 = null;
+        Map<UUID, PrimaryRequestState> mappings0 = null;
 
         int size = keys.size();
 
+        boolean mappingKnown = cctx.topology().rebalanceFinished(topVer) &&
+            !cctx.discovery().hasNearCache(cctx.cacheId(), topVer);
+
         try {
-            if (size == 1 && !fastMap) {
+            if (size == 1) {
                 assert remapKeys == null || remapKeys.size() == 1;
 
-                singleReq0 = mapSingleUpdate(topVer, futVer, updVer);
+                singleReq0 = mapSingleUpdate(topVer, futId, mappingKnown);
             }
             else {
-                Map<UUID, GridNearAtomicFullUpdateRequest> pendingMappings = mapUpdate(topNodes,
+                Map<UUID, PrimaryRequestState> pendingMappings = mapUpdate(topNodes,
                     topVer,
-                    futVer,
-                    updVer,
-                    remapKeys);
+                    futId,
+                    remapKeys,
+                    mappingKnown);
 
                 if (pendingMappings.size() == 1)
                     singleReq0 = F.firstValue(pendingMappings);
                 else {
-                    if (syncMode == PRIMARY_SYNC) {
-                        mappings0 = U.newHashMap(pendingMappings.size());
-
-                        for (GridNearAtomicFullUpdateRequest req : pendingMappings.values()) {
-                            if (req.hasPrimary())
-                                mappings0.put(req.nodeId(), req);
-                        }
-                    }
-                    else
-                        mappings0 = pendingMappings;
+                    mappings0 = pendingMappings;
 
                     assert !mappings0.isEmpty() || size == 0 : this;
                 }
             }
 
             synchronized (mux) {
-                assert this.futVer == futVer || (this.isDone() && this.error() != null);
-                assert this.topVer == topVer;
+                assert this.futId == null : this;
+                assert this.topVer == AffinityTopologyVersion.ZERO : this;
 
-                this.updVer = updVer;
+                this.topVer = topVer;
+                this.futId = futId;
 
                 resCnt = 0;
 
@@ -686,6 +814,12 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
 
                 this.remapKeys = null;
             }
+
+            if (storeFuture() && !cctx.mvcc().addAtomicFuture(futId, this)) {
+                assert isDone();
+
+                return;
+            }
         }
         catch (Exception e) {
             err = e;
@@ -699,56 +833,133 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
 
         // Optimize mapping for single key.
         if (singleReq0 != null)
-            mapSingle(singleReq0.nodeId(), singleReq0);
+            sendSingleRequest(singleReq0.req.nodeId(), singleReq0.req);
         else {
             assert mappings0 != null;
 
-            if (size == 0)
+            if (size == 0) {
                 onDone(new GridCacheReturn(cctx, true, true, null, true));
+
+                return;
+            }
             else
-                doUpdate(mappings0);
+                sendUpdateRequests(mappings0);
         }
+
+        if (syncMode == FULL_ASYNC) {
+            onDone(new GridCacheReturn(cctx, true, true, null, true));
+
+            return;
+        }
+
+        if (mappingKnown && syncMode == FULL_SYNC && cctx.discovery().topologyVersion() != topVer.topologyVersion())
+            checkDhtNodes(futId);
     }
 
-    /**
-     * @return Future version.
-     */
-    private GridCacheVersion onFutureDone() {
-        GridCacheVersion ver0;
+    private void checkDhtNodes(Long futId) {
+        GridCacheReturn opRes0 = null;
+        CachePartialUpdateCheckedException err0 = null;
+        AffinityTopologyVersion remapTopVer0 = null;
+
+        List<GridNearAtomicCheckUpdateRequest> checkReqs = null;
 
-        GridFutureAdapter<Void> fut0;
+        boolean rcvAll = false;
 
         synchronized (mux) {
-            fut0 = topCompleteFut;
+            if (this.futId == null || !this.futId.equals(futId))
+                return;
+
+            if (singleReq != null) {
+                if (!singleReq.req.initMappingLocally())
+                    return;
+
+                DhtLeftResult res = singleReq.checkDhtNodes(cctx);
+
+                if (res == DhtLeftResult.DONE) {
+                    opRes0 = opRes;
+                    err0 = err;
+                    remapTopVer0 = onAllReceived();
+                }
+                else if (res == DhtLeftResult.ALL_RCVD_CHECK_PRIMARY)
+                    checkReqs = Collections.singletonList(new GridNearAtomicCheckUpdateRequest(singleReq.req));
+                else
+                    return;
+            }
+            else {
+                if (mappings != null) {
+                    for (PrimaryRequestState reqState : mappings.values()) {
+                        if (!reqState.req.initMappingLocally())
+                            continue;
+
+                        DhtLeftResult res = reqState.checkDhtNodes(cctx);
+
+                        if (res == DhtLeftResult.DONE) {
+                            assert mappings.size() > resCnt : "[mappings=" + mappings.size() + ", cnt=" + resCnt + ']';
+
+                            resCnt++;
 
-            topCompleteFut = null;
+                            if (mappings.size() == resCnt) {
+                                rcvAll = true;
 
-            ver0 = futVer;
+                                opRes0 = opRes;
+                                err0 = err;
+                                remapTopVer0 = onAllReceived();
 
-            futVer = null;
+                                break;
+                            }
+                        }
+                        else if (res == DhtLeftResult.ALL_RCVD_CHECK_PRIMARY) {
+                            if (checkReqs == null)
+                                checkReqs = new ArrayList<>(mappings.size());
+
+                            checkReqs.add(new GridNearAtomicCheckUpdateRequest(reqState.req));
+                        }
+                    }
+                }
+                else
+                    return;
+            }
         }
 
-        if (fut0 != null)
-            fut0.onDone();
+        if (checkReqs != null) {
+            assert !rcvAll;
+
+            for (int i = 0; i < checkReqs.size(); i++)
+                sendCheckUpdateRequest(checkReqs.get(i));
+        }
+        else if (rcvAll)
+            finishUpdateFuture(opRes0, err0, remapTopVer0);
+    }
+
+    /**
+     * @return Future version.
+     */
+    private Long onFutureDone() {
+        Long id0;
+
+        synchronized (mux) {
+            id0 = futId;
+
+            futId = null;
+        }
 
-        return ver0;
+        return id0;
     }
 
     /**
      * @param topNodes Cache nodes.
      * @param topVer Topology version.
-     * @param futVer Future version.
-     * @param updVer Update version.
+     * @param futId Future ID.
      * @param remapKeys Keys to remap.
      * @return Mapping.
      * @throws Exception If failed.
      */
     @SuppressWarnings("ForLoopReplaceableByForEach")
-    private Map<UUID, GridNearAtomicFullUpdateRequest> mapUpdate(Collection<ClusterNode> topNodes,
+    private Map<UUID, PrimaryRequestState> mapUpdate(Collection<ClusterNode> topNodes,
         AffinityTopologyVersion topVer,
-        GridCacheVersion futVer,
-        @Nullable GridCacheVersion updVer,
-        @Nullable Collection<KeyCacheObject> remapKeys) throws Exception {
+        Long futId,
+        @Nullable Collection<KeyCacheObject> remapKeys,
+        boolean mappingKnown) throws Exception {
         Iterator<?> it = null;
 
         if (vals != null)
@@ -764,7 +975,7 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
         if (conflictRmvVals != null)
             conflictRmvValsIt = conflictRmvVals.iterator();
 
-        Map<UUID, GridNearAtomicFullUpdateRequest> pendingMappings = U.newHashMap(topNodes.size());
+        Map<UUID, PrimaryRequestState> pendingMappings = U.newHashMap(topNodes.size());
 
         // Create mappings first, then send messages.
         for (Object key : keys) {
@@ -819,55 +1030,50 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
             else
                 val = EntryProcessorResourceInjectorProxy.wrap(cctx.kernalContext(), (EntryProcessor)val);
 
-            List<ClusterNode> affNodes = mapKey(cacheKey, topVer);
+            List<ClusterNode> nodes = cctx.affinity().nodesByKey(cacheKey, topVer);
 
-            if (affNodes.isEmpty())
+            if (F.isEmpty(nodes))
                 throw new ClusterTopologyServerNotFoundException("Failed to map keys for cache " +
                     "(all partition nodes left the grid).");
 
-            int i = 0;
-
-            for (int n = 0; n < affNodes.size(); n++) {
-                ClusterNode affNode = affNodes.get(n);
-
-                if (affNode == null)
-                    throw new ClusterTopologyServerNotFoundException("Failed to map keys for cache " +
-                        "(all partition nodes left the grid).");
-
-                UUID nodeId = affNode.id();
-
-                GridNearAtomicFullUpdateRequest mapped = pendingMappings.get(nodeId);
-
-                if (mapped == null) {
-                    mapped = new GridNearAtomicFullUpdateRequest(
-                        cctx.cacheId(),
-                        nodeId,
-                        futVer,
-                        fastMap,
-                        updVer,
-                        topVer,
-                        topLocked,
-                        syncMode,
-                        op,
-                        retval,
-                        expiryPlc,
-                        invokeArgs,
-                        filter,
-                        subjId,
-                        taskNameHash,
-                        skipStore,
-                        keepBinary,
-                        cctx.kernalContext().clientNode(),
-                        cctx.deploymentEnabled(),
-                        keys.size());
-
-                    pendingMappings.put(nodeId, mapped);
-                }
+            ClusterNode primary = nodes.get(0);
+
+            boolean needPrimaryRes = !mappingKnown || primary.isLocal();
+
+            UUID nodeId = primary.id();
 
-                mapped.addUpdateEntry(cacheKey, val, conflictTtl, conflictExpireTime, conflictVer, i == 0);
+            PrimaryRequestState mapped = pendingMappings.get(nodeId);
 
-                i++;
+            if (mapped == null) {
+                GridNearAtomicFullUpdateRequest req = new GridNearAtomicFullUpdateRequest(
+                    cctx.cacheId(),
+                    nodeId,
+                    futId,
+                    topVer,
+                    topLocked,
+                    syncMode,
+                    op,
+                    retval,
+                    expiryPlc,
+                    invokeArgs,
+                    filter,
+                    subjId,
+                    taskNameHash,
+                    needPrimaryRes,
+                    skipStore,
+                    keepBinary,
+                    cctx.deploymentEnabled(),
+                    keys.size());
+
+                mapped = new PrimaryRequestState(req, nodes, false);
+
+                pendingMappings.put(nodeId, mapped);
             }
+
+            if (mapped.req.initMappingLocally())
+                mapped.addMapping(nodes);
+
+            mapped.req.addUpdateEntry(cacheKey, val, conflictTtl, conflictExpireTime, conflictVer);
         }
 
         return pendingMappings;
@@ -875,14 +1081,13 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
 
     /**
      * @param topVer Topology version.
-     * @param futVer Future version.
-     * @param updVer Update version.
+     * @param futId Future ID.
+     * @param mappingKnown {@code True} if update mapping is known locally.
      * @return Request.
      * @throws Exception If failed.
      */
-    private GridNearAtomicFullUpdateRequest mapSingleUpdate(AffinityTopologyVersion topVer,
-        GridCacheVersion futVer,
-        @Nullable GridCacheVersion updVer) throws Exception {
+    private PrimaryRequestState mapSingleUpdate(AffinityTopologyVersion topVer, Long futId, boolean mappingKnown)
+        throws Exception {
         Object key = F.first(keys);
 
         Object val;
@@ -935,18 +1140,20 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
         else
             val = EntryProcessorResourceInjectorProxy.wrap(cctx.kernalContext(), (EntryProcessor)val);
 
-        ClusterNode primary = cctx.affinity().primaryByPartition(cacheKey.partition(), topVer);
+        List<ClusterNode> nodes = cctx.affinity().nodesByKey(cacheKey, topVer);
 
-        if (primary == null)
-            throw new ClusterTopologyServerNotFoundException("Failed to map keys for cache (all partition nodes " +
-                "left the grid).");
+        if (F.isEmpty(nodes))
+            throw new ClusterTopologyServerNotFoundException("Failed to map keys for cache " +
+                    "(all partition nodes left the grid).");
+
+        ClusterNode primary = nodes.get(0);
+
+        boolean needPrimaryRes = !mappingKnown || primary.isLocal() || nodes.size() == 1;
 
         GridNearAtomicFullUpdateRequest req = new GridNearAtomicFullUpdateRequest(
             cctx.cacheId(),
             primary.id(),
-            futVer,
-            fastMap,
-            updVer,
+            futId,
             topVer,
             topLocked,
             syncMode,
@@ -957,9 +1164,9 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
             filter,
             subjId,
             taskNameHash,
+            needPrimaryRes,
             skipStore,
             keepBinary,
-            cctx.kernalContext().clientNode(),
             cctx.deploymentEnabled(),
             1);
 
@@ -967,26 +1174,9 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
             val,
             conflictTtl,
             conflictExpireTime,
-            conflictVer,
-            true);
-
-        return req;
-    }
-
-    /**
-     * Maps key to nodes. If filters are absent and operation is not TRANSFORM, then we can assign version on near
-     * node and send updates in parallel to all participating nodes.
-     *
-     * @param key Key to map.
-     * @param topVer Topology version to map.
-     * @return Collection of nodes to which key is mapped.
-     */
-    private List<ClusterNode> mapKey(KeyCacheObject key, AffinityTopologyVersion topVer) {
-        GridCacheAffinityManager affMgr = cctx.affinity();
+            conflictVer);
 
-        // If we can send updates in parallel - do it.
-        return fastMap ? cctx.topology().nodes(affMgr.partition(key), topVer) :
-            Collections.singletonList(affMgr.primaryByKey(key, topVer));
+        return new PrimaryRequestState(req, nodes, true);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
index 22e01ae..4e20fc7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
@@ -23,11 +23,11 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.UUID;
-import java.util.concurrent.ConcurrentLinkedQueue;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.GridDirectCollection;
 import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheDeployable;
@@ -39,7 +39,6 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.GridLongList;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
@@ -59,29 +58,18 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
     @GridDirectTransient
     private UUID nodeId;
 
-    /** Future version. */
-    private GridCacheVersion futVer;
+    /** Future ID. */
+    private long futId;
 
-    /** Update error. */
-    @GridDirectTransient
-    private volatile IgniteCheckedException err;
-
-    /** Serialized error. */
-    private byte[] errBytes;
+    /** */
+    private UpdateErrors errs;
 
     /** Return value. */
     @GridToStringInclude
     private GridCacheReturn ret;
 
-    /** Failed keys. */
-    @GridToStringInclude
-    @GridDirectCollection(KeyCacheObject.class)
-    private volatile Collection<KeyCacheObject> failedKeys;
-
-    /** Keys that should be remapped. */
-    @GridToStringInclude
-    @GridDirectCollection(KeyCacheObject.class)
-    private List<KeyCacheObject> remapKeys;
+    /** */
+    private AffinityTopologyVersion remapTopVer;
 
     /** Indexes of keys for which values were generated on primary node (used if originating node has near cache). */
     @GridDirectCollection(int.class)
@@ -108,6 +96,15 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
     /** Partition ID. */
     private int partId = -1;
 
+    /** */
+    @GridDirectCollection(UUID.class)
+    @GridToStringInclude
+    private List<UUID> dhtNodes;
+
+    /** */
+    @GridDirectTransient
+    private boolean nodeLeft;
+
     /**
      * Empty constructor required by {@link Externalizable}.
      */
@@ -118,24 +115,52 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
     /**
      * @param cacheId Cache ID.
      * @param nodeId Node ID this reply should be sent to.
-     * @param futVer Future version.
+     * @param futId Future ID.
+     * @param partId Partition.
+     * @param nodeLeft {@code True} if primary node failed.
      * @param addDepInfo Deployment info flag.
      */
-    public GridNearAtomicUpdateResponse(int cacheId, UUID nodeId, GridCacheVersion futVer, boolean addDepInfo) {
-        assert futVer != null;
-
+    public GridNearAtomicUpdateResponse(int cacheId,
+        UUID nodeId,
+        long futId,
+        int partId,
+        boolean nodeLeft,
+        boolean addDepInfo) {
         this.cacheId = cacheId;
         this.nodeId = nodeId;
-        this.futVer = futVer;
+        this.futId = futId;
+        this.partId = partId;
+        this.nodeLeft = nodeLeft;
         this.addDepInfo = addDepInfo;
     }
 
+    /**
+     * @return {@code True} if primary node failed.
+     */
+    public boolean nodeLeftResponse() {
+        return nodeLeft;
+    }
+
     /** {@inheritDoc} */
     @Override public int lookupIndex() {
         return CACHE_MSG_IDX;
     }
 
     /**
+     * @param dhtNodes DHT nodes.
+     */
+    public void dhtNodes(List<UUID> dhtNodes) {
+        this.dhtNodes = dhtNodes;
+    }
+
+    /**
+     * @return DHT nodes.
+     */
+    @Nullable public List<UUID> dhtNodes() {
+        return dhtNodes;
+    }
+
+    /**
      * @return Node ID this response should be sent to.
      */
     public UUID nodeId() {
@@ -150,17 +175,10 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
     }
 
     /**
-     * @return Future version.
+     * @return Future ID.
      */
-    public GridCacheVersion futureVersion() {
-        return futVer;
-    }
-
-    /**
-     * @param partId Partition ID for proper striping on near node.
-     */
-    public void partition(int partId) {
-        this.partId = partId;
+    public long futureId() {
+        return futId;
     }
 
     /**
@@ -169,19 +187,22 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
      * @param err Error.
      */
     public void error(IgniteCheckedException err){
-        this.err = err;
+        if (errs == null)
+            errs = new UpdateErrors();
+
+        errs.onError(err);
     }
 
     /** {@inheritDoc} */
     @Override public IgniteCheckedException error() {
-        return err;
+        return errs != null ? errs.error() : null;
     }
 
     /**
      * @return Collection of failed keys.
      */
     public Collection<KeyCacheObject> failedKeys() {
-        return failedKeys;
+        return errs != null ? errs.failedKeys() : null;
     }
 
     /**
@@ -200,17 +221,17 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
     }
 
     /**
-     * @param remapKeys Remap keys.
+     * @param remapTopVer Topology version to remap update.
      */
-    public void remapKeys(List<KeyCacheObject> remapKeys) {
-        this.remapKeys = remapKeys;
+    void remapTopologyVersion(AffinityTopologyVersion remapTopVer) {
+        this.remapTopVer = remapTopVer;
     }
 
     /**
-     * @return Remap keys.
+     * @return Topology version if update should be remapped.
      */
-    public Collection<KeyCacheObject> remapKeys() {
-        return remapKeys;
+    @Nullable AffinityTopologyVersion remapTopologyVersion() {
+        return remapTopVer;
     }
 
     /**
@@ -221,7 +242,7 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
      * @param ttl TTL for near cache update.
      * @param expireTime Expire time for near cache update.
      */
-    public void addNearValue(int keyIdx,
+    void addNearValue(int keyIdx,
         @Nullable CacheObject val,
         long ttl,
         long expireTime) {
@@ -242,7 +263,7 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
      * @param expireTime Expire time for near cache update.
      */
     @SuppressWarnings("ForLoopReplaceableByForEach")
-    public void addNearTtl(int keyIdx, long ttl, long expireTime) {
+    void addNearTtl(int keyIdx, long ttl, long expireTime) {
         if (ttl >= 0) {
             if (nearTtls == null) {
                 nearTtls = new GridLongList(16);
@@ -299,7 +320,7 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
     /**
      * @param nearVer Version generated on primary node to be used for originating node's near cache update.
      */
-    public void nearVersion(GridCacheVersion nearVer) {
+    void nearVersion(GridCacheVersion nearVer) {
         this.nearVer = nearVer;
     }
 
@@ -313,7 +334,7 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
     /**
      * @param keyIdx Index of key for which update was skipped
      */
-    public void addSkippedIndex(int keyIdx) {
+    void addSkippedIndex(int keyIdx) {
         if (nearSkipIdxs == null)
             nearSkipIdxs = new ArrayList<>();
 
@@ -351,35 +372,10 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
      * @param e Error cause.
      */
     public synchronized void addFailedKey(KeyCacheObject key, Throwable e) {
-        if (failedKeys == null)
-            failedKeys = new ConcurrentLinkedQueue<>();
-
-        failedKeys.add(key);
-
-        if (err == null)
-            err = new IgniteCheckedException("Failed to update keys on primary node.");
-
-        err.addSuppressed(e);
-    }
-
-    /**
-     * Adds keys to collection of failed keys.
-     *
-     * @param keys Key to add.
-     * @param e Error cause.
-     */
-    public synchronized void addFailedKeys(Collection<KeyCacheObject> keys, Throwable e) {
-        if (keys != null) {
-            if (failedKeys == null)
-                failedKeys = new ArrayList<>(keys.size());
-
-            failedKeys.addAll(keys);
-        }
-
-        if (err == null)
-            err = new IgniteCheckedException("Failed to update keys on primary node.");
+        if (errs == null)
+            errs = new UpdateErrors();
 
-        err.addSuppressed(e);
+        errs.addFailedKey(key, e);
     }
 
     /**
@@ -387,18 +383,12 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
      *
      * @param keys Key to add.
      * @param e Error cause.
-     * @param ctx Context.
      */
-    public synchronized void addFailedKeys(Collection<KeyCacheObject> keys, Throwable e, GridCacheContext ctx) {
-        if (failedKeys == null)
-            failedKeys = new ArrayList<>(keys.size());
+    synchronized void addFailedKeys(Collection<KeyCacheObject> keys, Throwable e) {
+        if (errs == null)
+            errs = new UpdateErrors();
 
-        failedKeys.addAll(keys);
-
-        if (err == null)
-            err = new IgniteCheckedException("Failed to update keys on primary node.");
-
-        err.addSuppressed(e);
+        errs.addFailedKeys(keys, e);
     }
 
     /** {@inheritDoc}
@@ -406,14 +396,10 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
     @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException {
         super.prepareMarshal(ctx);
 
-        if (err != null && errBytes == null)
-            errBytes = U.marshal(ctx, err);
-
         GridCacheContext cctx = ctx.cacheContext(cacheId);
 
-        prepareMarshalCacheObjects(failedKeys, cctx);
-
-        prepareMarshalCacheObjects(remapKeys, cctx);
+        if (errs != null)
+            errs.prepareMarshal(this, cctx);
 
         prepareMarshalCacheObjects(nearVals, cctx);
 
@@ -425,14 +411,10 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
     @Override public void finishUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException {
         super.finishUnmarshal(ctx, ldr);
 
-        if (errBytes != null && err == null)
-            err = U.unmarshal(ctx, errBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
-
         GridCacheContext cctx = ctx.cacheContext(cacheId);
 
-        finishUnmarshalCacheObjects(failedKeys, cctx, ldr);
-
-        finishUnmarshalCacheObjects(remapKeys, cctx, ldr);
+        if (errs != null)
+            errs.finishUnmarshal(this, cctx, ldr);
 
         finishUnmarshalCacheObjects(nearVals, cctx, ldr);
 
@@ -471,19 +453,19 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
 
         switch (writer.state()) {
             case 3:
-                if (!writer.writeByteArray("errBytes", errBytes))
+                if (!writer.writeCollection("dhtNodes", dhtNodes, MessageCollectionItemType.UUID))
                     return false;
 
                 writer.incrementState();
 
             case 4:
-                if (!writer.writeCollection("failedKeys", failedKeys, MessageCollectionItemType.MSG))
+                if (!writer.writeMessage("errs", errs))
                     return false;
 
                 writer.incrementState();
 
             case 5:
-                if (!writer.writeMessage("futVer", futVer))
+                if (!writer.writeLong("futId", futId))
                     return false;
 
                 writer.incrementState();
@@ -531,7 +513,7 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
                 writer.incrementState();
 
             case 13:
-                if (!writer.writeCollection("remapKeys", remapKeys, MessageCollectionItemType.MSG))
+                if (!writer.writeMessage("remapTopVer", remapTopVer))
                     return false;
 
                 writer.incrementState();
@@ -559,7 +541,7 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
 
         switch (reader.state()) {
             case 3:
-                errBytes = reader.readByteArray("errBytes");
+                dhtNodes = reader.readCollection("dhtNodes", MessageCollectionItemType.UUID);
 
                 if (!reader.isLastRead())
                     return false;
@@ -567,7 +549,7 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
                 reader.incrementState();
 
             case 4:
-                failedKeys = reader.readCollection("failedKeys", MessageCollectionItemType.MSG);
+                errs = reader.readMessage("errs");
 
                 if (!reader.isLastRead())
                     return false;
@@ -575,7 +557,7 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
                 reader.incrementState();
 
             case 5:
-                futVer = reader.readMessage("futVer");
+                futId = reader.readLong("futId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -639,7 +621,7 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
                 reader.incrementState();
 
             case 13:
-                remapKeys = reader.readCollection("remapKeys", MessageCollectionItemType.MSG);
+                remapTopVer = reader.readMessage("remapTopVer");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/UpdateErrors.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/UpdateErrors.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/UpdateErrors.java
new file mode 100644
index 0000000..1d415c8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/UpdateErrors.java
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.GridDirectCollection;
+import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheMessage;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ *
+ */
+public class UpdateErrors implements Message {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Failed keys. */
+    @GridToStringInclude
+    @GridDirectCollection(KeyCacheObject.class)
+    private List<KeyCacheObject> failedKeys;
+
+    /** Update error. */
+    @GridDirectTransient
+    @GridToStringInclude
+    private IgniteCheckedException err;
+
+    /** Serialized update error. */
+    private byte[] errBytes;
+
+    /**
+     *
+     */
+    public UpdateErrors() {
+        // No-op.
+    }
+
+    /**
+     * @param err Error.
+     */
+    public UpdateErrors(IgniteCheckedException err) {
+        assert err != null;
+
+        this.err = err;
+    }
+
+    /**
+     * @param err Error.
+     */
+    public void onError(IgniteCheckedException err){
+        this.err = err;
+    }
+
+    /**
+     * @return Error.
+     */
+    public IgniteCheckedException error() {
+        return err;
+    }
+
+    /**
+     * @return Failed keys.
+     */
+    public Collection<KeyCacheObject> failedKeys() {
+        return failedKeys;
+    }
+
+    /**
+     * Adds key to collection of failed keys.
+     *
+     * @param key Key to add.
+     * @param e Error cause.
+     */
+    void addFailedKey(KeyCacheObject key, Throwable e) {
+        if (failedKeys == null)
+            failedKeys = new ArrayList<>();
+
+        failedKeys.add(key);
+
+        if (err == null)
+            err = new IgniteCheckedException("Failed to update keys.");
+
+        err.addSuppressed(e);
+    }
+
+    /**
+     * @param keys Keys.
+     * @param e Error.
+     */
+    void addFailedKeys(Collection<KeyCacheObject> keys, Throwable e) {
+        if (failedKeys == null)
+            failedKeys = new ArrayList<>(keys.size());
+
+        failedKeys.addAll(keys);
+
+        if (err == null)
+            err = new IgniteCheckedException("Failed to update keys on primary node.");
+
+        err.addSuppressed(e);
+    }
+
+    /** {@inheritDoc} */
+    void prepareMarshal(GridCacheMessage msg, GridCacheContext cctx) throws IgniteCheckedException {
+        msg.prepareMarshalCacheObjects(failedKeys, cctx);
+
+        if (errBytes == null)
+            errBytes = U.marshal(cctx.marshaller(), err);
+    }
+
+    /** {@inheritDoc} */
+    void finishUnmarshal(GridCacheMessage msg, GridCacheContext cctx, ClassLoader ldr) throws IgniteCheckedException {
+        msg.finishUnmarshalCacheObjects(failedKeys, cctx, ldr);
+
+        if (errBytes != null && err == null)
+            err = U.unmarshal(cctx.marshaller(), errBytes, U.resolveClassLoader(ldr, cctx.gridConfig()));
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeByteArray("errBytes", errBytes))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeCollection("failedKeys", failedKeys, MessageCollectionItemType.MSG))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                errBytes = reader.readByteArray("errBytes");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                failedKeys = reader.readCollection("failedKeys", MessageCollectionItemType.MSG);
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(UpdateErrors.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -46;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 2;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(UpdateErrors.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
index 41632ef..62aecd1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import java.io.Externalizable;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
@@ -43,7 +44,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheA
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache;
-import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateResponse;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicNearResponse;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse;
 import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo;
@@ -141,10 +142,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
         List<Integer> nearValsIdxs = res.nearValuesIndexes();
         List<Integer> skipped = res.skippedIndexes();
 
-        GridCacheVersion ver = req.updateVersion();
-
-        if (ver == null)
-            ver = res.nearVersion();
+        GridCacheVersion ver = res.nearVersion();
 
         assert ver != null : "Failed to find version [req=" + req + ", res=" + res + ']';
 
@@ -194,7 +192,6 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
                 processNearAtomicUpdateResponse(ver,
                     key,
                     val,
-                    null,
                     ttl,
                     expireTime,
                     req.keepBinary(),
@@ -212,7 +209,6 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
      * @param ver Version.
      * @param key Key.
      * @param val Value.
-     * @param valBytes Value bytes.
      * @param ttl TTL.
      * @param expireTime Expire time.
      * @param nodeId Node ID.
@@ -224,7 +220,6 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
         GridCacheVersion ver,
         KeyCacheObject key,
         @Nullable CacheObject val,
-        @Nullable byte[] valBytes,
         long ttl,
         long expireTime,
         boolean keepBinary,
@@ -241,7 +236,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
                 try {
                     entry = entryEx(key, topVer);
 
-                    GridCacheOperation op = (val != null || valBytes != null) ? UPDATE : DELETE;
+                    GridCacheOperation op = val != null ? UPDATE : DELETE;
 
                     GridCacheUpdateAtomicResult updRes = entry.innerUpdate(
                         ver,
@@ -299,11 +294,12 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
      * @param nodeId Sender node ID.
      * @param req Dht atomic update request.
      * @param res Dht atomic update response.
+     * @return Evicted near keys (if any).
      */
-    public void processDhtAtomicUpdateRequest(
+    @Nullable public List<KeyCacheObject> processDhtAtomicUpdateRequest(
         UUID nodeId,
         GridDhtAtomicAbstractUpdateRequest req,
-        GridDhtAtomicUpdateResponse res
+        GridDhtAtomicNearResponse res
     ) {
         GridCacheVersion ver = req.writeVersion();
 
@@ -313,6 +309,8 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
 
         String taskName = ctx.kernalContext().task().resolveTaskName(req.taskNameHash());
 
+        List<KeyCacheObject> nearEvicted = null;
+
         for (int i = 0; i < req.nearSize(); i++) {
             KeyCacheObject key = req.nearKey(i);
 
@@ -322,7 +320,10 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
                         GridCacheEntryEx entry = peekEx(key);
 
                         if (entry == null) {
-                            res.addNearEvicted(key);
+                            if (nearEvicted == null)
+                                nearEvicted = new ArrayList<>();
+
+                            nearEvicted.add(key);
 
                             break;
                         }
@@ -388,6 +389,8 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
                 res.addFailedKey(key, new IgniteCheckedException("Failed to update near cache key: " + key, e));
             }
         }
+
+        return nearEvicted;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
index b3f0684..485059f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
@@ -425,7 +425,7 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
 
                                     onEntryUpdate(evt, notify, loc, recordIgniteEvt);
                                 }
-                            });
+                            }, sync);
                         }
                     }
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
index 5ca3da8..35fbe11 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
@@ -140,7 +140,10 @@ public interface IgniteCacheObjectProcessor extends GridProcessor {
      *        before stored in cache.
      * @return Cache key object.
      */
-    public KeyCacheObject toCacheKeyObject(CacheObjectContext ctx, @Nullable GridCacheContext cctx, Object obj, boolean userObj);
+    public KeyCacheObject toCacheKeyObject(CacheObjectContext ctx,
+        @Nullable GridCacheContext cctx,
+        Object obj,
+        boolean userObj);
 
     /**
      * @param ctx Cache context.

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
index ff7c4ba..e0549fb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
@@ -231,8 +231,8 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
                 cctx.affinity().partition(obj, false) :
                 ctx.kernalContext().affinity().partition0(ctx.cacheName(), obj, null);
         }
-        catch (IgniteCheckedException ignored) {
-            U.error(log, "Failed to get partition");
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to get partition", e);
 
             return  -1;
         }
@@ -327,13 +327,7 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
 
         /**
          * @param key Key.
-         */
-        UserKeyCacheObjectImpl(Object key) {
-            this(key, -1);
-        }
-
-        /**
-         * @param key Key.
+         * @param part Partition.
          */
         UserKeyCacheObjectImpl(Object key, int part) {
             super(key, null, part);
@@ -341,6 +335,8 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
 
         /**
          * @param key Key.
+         * @param valBytes Marshalled key.
+         * @param part Partition.
          */
         UserKeyCacheObjectImpl(Object key, byte[] valBytes, int part) {
             super(key, valBytes, part);
@@ -366,10 +362,10 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
 
                     Object val = ctx.processor().unmarshal(ctx, valBytes, ldr);
 
-                    return new KeyCacheObjectImpl(val, valBytes);
+                    return new KeyCacheObjectImpl(val, valBytes, partition());
                 }
 
-                return new KeyCacheObjectImpl(val, valBytes);
+                return new KeyCacheObjectImpl(val, valBytes, partition());
             }
             catch (IgniteCheckedException e) {
                 throw new IgniteException("Failed to marshal object: " + val, e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java
index 53096ab..6c85b32 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java
@@ -56,7 +56,7 @@ public class StripedExecutor implements ExecutorService {
 
     /**
      * @param cnt Count.
-     * @param gridName Node name.
+     * @param igniteInstanceName Node name.
      * @param poolName Pool name.
      * @param log Logger.
      */
@@ -435,7 +435,11 @@ public class StripedExecutor implements ExecutorService {
          * Starts the stripe.
          */
         void start() {
-            thread = new IgniteThread(igniteInstanceName, poolName + "-stripe-" + idx, this);
+            thread = new IgniteThread(igniteInstanceName,
+                poolName + "-stripe-" + idx,
+                this,
+                IgniteThread.GRP_IDX_UNASSIGNED,
+                idx);
 
             thread.start();
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
index 7abd367..96f3797 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
@@ -18,8 +18,10 @@
 package org.apache.ignite.internal.util.future;
 
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.IgniteFutureCancelledCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -118,7 +120,14 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> implements Ig
         }
         catch (IgniteCheckedException e) {
             if (!ignoreFailure(e)) {
-                U.error(null, "Failed to execute compound future reducer: " + this, e);
+                if (e instanceof NodeStoppingException) {
+                    IgniteLogger log = logger();
+
+                    if (log != null && log.isDebugEnabled())
+                        log.debug("Failed to execute compound future reducer, node stopped.");
+                }
+                else
+                    U.error(null, "Failed to execute compound future reducer: " + this, e);
 
                 onDone(e);
             }


[5/8] ignite git commit: ignite-4705 Atomic cache protocol change: notify client node from backups

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
index c92e0f5..39abb73 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
@@ -17,12 +17,22 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicReference;
 import javax.cache.expiry.ExpiryPolicy;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheEntryPredicate;
 import org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException;
@@ -31,16 +41,19 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheMvccManager;
 import org.apache.ignite.internal.processors.cache.GridCacheOperation;
 import org.apache.ignite.internal.processors.cache.GridCacheReturn;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
-import org.apache.ignite.internal.util.typedef.CI2;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
 
 /**
  * Base for near atomic update futures.
@@ -108,28 +121,24 @@ public abstract class GridNearAtomicAbstractUpdateFuture extends GridFutureAdapt
     protected boolean topLocked;
 
     /** Remap count. */
+    @GridToStringInclude
     protected int remapCnt;
 
     /** Current topology version. */
+    @GridToStringInclude
     protected AffinityTopologyVersion topVer = AffinityTopologyVersion.ZERO;
 
     /** */
-    protected GridCacheVersion updVer;
-
-    /** Topology version when got mapping error. */
-    protected AffinityTopologyVersion mapErrTopVer;
-
-    /** */
-    protected int resCnt;
+    @GridToStringInclude
+    protected AffinityTopologyVersion remapTopVer;
 
     /** Error. */
+    @GridToStringInclude
     protected CachePartialUpdateCheckedException err;
 
     /** Future ID. */
-    protected GridCacheVersion futVer;
-
-    /** Completion future for a particular topology version. */
-    protected GridFutureAdapter<Void> topCompleteFut;
+    @GridToStringInclude
+    protected Long futId;
 
     /** Operation result. */
     protected GridCacheReturn opRes;
@@ -198,10 +207,30 @@ public abstract class GridNearAtomicAbstractUpdateFuture extends GridFutureAdapt
         this.remapCnt = remapCnt;
     }
 
+    /** {@inheritDoc} */
+    @Override public final IgniteInternalFuture<Void> completeFuture(AffinityTopologyVersion topVer) {
+        return null;
+    }
+
+    /**
+     * @param req Request.
+     */
+    void sendCheckUpdateRequest(GridNearAtomicCheckUpdateRequest req) {
+        try {
+            cctx.io().send(req.updateRequest().nodeId(), req, cctx.ioPolicy());
+        }
+        catch (ClusterTopologyCheckedException e) {
+            onSendError(req, e);
+        }
+        catch (IgniteCheckedException e) {
+            onDone(e);
+        }
+    }
+
     /**
      * Performs future mapping.
      */
-    public void map() {
+    public final void map() {
         AffinityTopologyVersion topVer = cctx.shared().lockedTopologyVersion(null);
 
         if (topVer == null)
@@ -212,18 +241,14 @@ public abstract class GridNearAtomicAbstractUpdateFuture extends GridFutureAdapt
             // Cannot remap.
             remapCnt = 1;
 
-            GridCacheVersion futVer = addAtomicFuture(topVer);
-
-            if (futVer != null)
-                map(topVer, futVer);
+            map(topVer);
         }
     }
 
     /**
      * @param topVer Topology version.
-     * @param futVer Future version
      */
-    protected abstract void map(AffinityTopologyVersion topVer, GridCacheVersion futVer);
+    protected abstract void map(AffinityTopologyVersion topVer);
 
     /**
      * Maps future on ready topology.
@@ -248,8 +273,8 @@ public abstract class GridNearAtomicAbstractUpdateFuture extends GridFutureAdapt
     /**
      * @return {@code True} future is stored by {@link GridCacheMvccManager#addAtomicFuture}.
      */
-    protected boolean storeFuture() {
-        return cctx.config().getAtomicWriteOrderMode() == CLOCK || syncMode != FULL_ASYNC;
+    final boolean storeFuture() {
+        return syncMode != FULL_ASYNC;
     }
 
     /**
@@ -258,12 +283,15 @@ public abstract class GridNearAtomicAbstractUpdateFuture extends GridFutureAdapt
      * @param nodeId Node ID.
      * @param req Request.
      */
-    protected void mapSingle(UUID nodeId, GridNearAtomicAbstractUpdateRequest req) {
+    final void sendSingleRequest(UUID nodeId, GridNearAtomicAbstractUpdateRequest req) {
         if (cctx.localNodeId().equals(nodeId)) {
             cache.updateAllAsyncInternal(nodeId, req,
-                new CI2<GridNearAtomicAbstractUpdateRequest, GridNearAtomicUpdateResponse>() {
+                new GridDhtAtomicCache.UpdateReplyClosure() {
                     @Override public void apply(GridNearAtomicAbstractUpdateRequest req, GridNearAtomicUpdateResponse res) {
-                        onResult(res.nodeId(), res, false);
+                        if (syncMode != FULL_ASYNC)
+                            onPrimaryResponse(res.nodeId(), res, false);
+                        else if (res.remapTopologyVersion() != null)
+                            ((GridDhtAtomicCache)cctx.cache()).remapToNewPrimary(req);
                     }
                 });
         }
@@ -272,18 +300,13 @@ public abstract class GridNearAtomicAbstractUpdateFuture extends GridFutureAdapt
                 cctx.io().send(req.nodeId(), req, cctx.ioPolicy());
 
                 if (msgLog.isDebugEnabled()) {
-                    msgLog.debug("Near update fut, sent request [futId=" + req.futureVersion() +
-                        ", writeVer=" + req.updateVersion() +
+                    msgLog.debug("Near update fut, sent request [futId=" + req.futureId() +
                         ", node=" + req.nodeId() + ']');
                 }
-
-                if (syncMode == FULL_ASYNC)
-                    onDone(new GridCacheReturn(cctx, true, true, null, true));
             }
             catch (IgniteCheckedException e) {
                 if (msgLog.isDebugEnabled()) {
-                    msgLog.debug("Near update fut, failed to send request [futId=" + req.futureVersion() +
-                        ", writeVer=" + req.updateVersion() +
+                    msgLog.debug("Near update fut, failed to send request [futId=" + req.futureId() +
                         ", node=" + req.nodeId() +
                         ", err=" + e + ']');
                 }
@@ -300,46 +323,377 @@ public abstract class GridNearAtomicAbstractUpdateFuture extends GridFutureAdapt
      * @param res Update response.
      * @param nodeErr {@code True} if response was created on node failure.
      */
-    public abstract void onResult(UUID nodeId, GridNearAtomicUpdateResponse res, boolean nodeErr);
+    public abstract void onPrimaryResponse(UUID nodeId, GridNearAtomicUpdateResponse res, boolean nodeErr);
+
+    /**
+     * @param nodeId Node ID.
+     * @param res Response.
+     */
+    public abstract void onDhtResponse(UUID nodeId, GridDhtAtomicNearResponse res);
 
     /**
      * @param req Request.
-     * @param e Error.
+     * @param res Response.
      */
-    protected final void onSendError(GridNearAtomicAbstractUpdateRequest req, IgniteCheckedException e) {
-        synchronized (mux) {
-            GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(cctx.cacheId(),
-                req.nodeId(),
-                req.futureVersion(),
-                cctx.deploymentEnabled());
+    final void onPrimaryError(GridNearAtomicAbstractUpdateRequest req, GridNearAtomicUpdateResponse res) {
+        assert res.error() != null;
+
+        if (err == null)
+            err = new CachePartialUpdateCheckedException("Failed to update keys (retry update if possible).");
+
+        Collection<KeyCacheObject> keys0 = res.failedKeys() != null ? res.failedKeys() : req.keys();
+
+        Collection<Object> keys = new ArrayList<>(keys0.size());
 
-            res.addFailedKeys(req.keys(), e);
+        for (KeyCacheObject key : keys0)
+            keys.add(cctx.cacheObjectContext().unwrapBinaryIfNeeded(key, keepBinary, false));
 
-            onResult(req.nodeId(), res, true);
+        err.add(keys, res.error(), req.topologyVersion());
+    }
+
+    /**
+     * @param req Request.
+     * @return Response to notify about primary failure.
+     */
+    final GridNearAtomicUpdateResponse primaryFailedResponse(GridNearAtomicAbstractUpdateRequest req) {
+        assert req.response() == null : req;
+        assert req.nodeId() != null : req;
+
+        if (msgLog.isDebugEnabled()) {
+            msgLog.debug("Near update fut, node left [futId=" + req.futureId() +
+                ", node=" + req.nodeId() + ']');
         }
+
+        GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(cctx.cacheId(),
+            req.nodeId(),
+            req.futureId(),
+            req.partition(),
+            true,
+            cctx.deploymentEnabled());
+
+        ClusterTopologyCheckedException e = new ClusterTopologyCheckedException("Primary node left grid " +
+            "before response is received: " + req.nodeId());
+
+        e.retryReadyFuture(cctx.shared().nextAffinityReadyFuture(req.topologyVersion()));
+
+        res.addFailedKeys(req.keys(), e);
+
+        return res;
+    }
+
+    /**
+     * @param req Request.
+     * @param e Error.
+     */
+    final void onSendError(GridNearAtomicAbstractUpdateRequest req, IgniteCheckedException e) {
+        GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(cctx.cacheId(),
+            req.nodeId(),
+            req.futureId(),
+            req.partition(),
+            e instanceof ClusterTopologyCheckedException,
+            cctx.deploymentEnabled());
+
+        res.addFailedKeys(req.keys(), e);
+
+        onPrimaryResponse(req.nodeId(), res, true);
+    }
+
+    /**
+     * @param req Request.
+     * @param e Error.
+     */
+    private void onSendError(GridNearAtomicCheckUpdateRequest req, IgniteCheckedException e) {
+        GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(cctx.cacheId(),
+            req.updateRequest().nodeId(),
+            req.futureId(),
+            req.partition(),
+            e instanceof ClusterTopologyCheckedException,
+            cctx.deploymentEnabled());
+
+        res.addFailedKeys(req.updateRequest().keys(), e);
+
+        onPrimaryResponse(req.updateRequest().nodeId(), res, true);
     }
 
     /**
-     * Adds future prevents topology change before operation complete.
-     * Should be invoked before topology lock released.
      *
-     * @param topVer Topology version.
-     * @return Future version in case future added.
      */
-    protected final GridCacheVersion addAtomicFuture(AffinityTopologyVersion topVer) {
-        GridCacheVersion futVer = cctx.versions().next(topVer);
+    static class PrimaryRequestState {
+        /** */
+        final GridNearAtomicAbstractUpdateRequest req;
+
+        /** */
+        @GridToStringInclude
+        Set<UUID> dhtNodes;
+
+        /** */
+        @GridToStringInclude
+        private Set<UUID> rcvd;
+
+        /** */
+        private boolean hasRes;
+
+        /**
+         * @param req Request.
+         * @param nodes Affinity nodes.
+         * @param single {@code True} if created for sigle-key operation.
+         */
+        PrimaryRequestState(GridNearAtomicAbstractUpdateRequest req, List<ClusterNode> nodes, boolean single) {
+            assert req != null && req.nodeId() != null : req;
+
+            this.req = req;
+
+            if (req.initMappingLocally()) {
+                if (single) {
+                    if (nodes.size() > 1) {
+                        dhtNodes = U.newHashSet(nodes.size() - 1);
+
+                        for (int i = 1; i < nodes.size(); i++)
+                            dhtNodes.add(nodes.get(i).id());
+                    }
+                    else
+                        dhtNodes = Collections.emptySet();
+                }
+                else {
+                    dhtNodes = new HashSet<>();
+
+                    for (int i = 1; i < nodes.size(); i++)
+                        dhtNodes.add(nodes.get(i).id());
+                }
+            }
+        }
+
+        /**
+         * @return Primary node ID.
+         */
+        UUID primaryId() {
+            return req.nodeId();
+        }
+
+        /**
+         * @param nodes Nodes.
+         */
+        void addMapping(List<ClusterNode> nodes) {
+            assert req.initMappingLocally();
+
+            for (int i = 1; i < nodes.size(); i++)
+                dhtNodes.add(nodes.get(i).id());
+        }
+
+        /**
+         * @param cctx Context.
+         * @return Check result.
+         */
+        DhtLeftResult checkDhtNodes(GridCacheContext cctx) {
+            assert req.initMappingLocally() : req;
 
-        synchronized (mux) {
-            assert this.futVer == null : this;
-            assert this.topVer == AffinityTopologyVersion.ZERO : this;
+            if (finished())
+                return DhtLeftResult.NOT_DONE;
 
-            this.topVer = topVer;
-            this.futVer = futVer;
+            boolean finished = false;
+
+            for (Iterator<UUID> it = dhtNodes.iterator(); it.hasNext();) {
+                UUID nodeId = it.next();
+
+                if (!cctx.discovery().alive(nodeId)) {
+                    it.remove();
+
+                    if (finished()) {
+                        finished = true;
+
+                        break;
+                    }
+                }
+            }
+
+            if (finished)
+                return DhtLeftResult.DONE;
+
+            if (dhtNodes.isEmpty())
+                return !req.needPrimaryResponse() ? DhtLeftResult.ALL_RCVD_CHECK_PRIMARY : DhtLeftResult.NOT_DONE;
+
+            return DhtLeftResult.NOT_DONE;
+        }
+
+        /**
+         * @return {@code True} if all expected responses are received.
+         */
+        private boolean finished() {
+            if (req.writeSynchronizationMode() == PRIMARY_SYNC)
+                return hasRes;
+
+            return (dhtNodes != null && dhtNodes.isEmpty()) && hasRes;
+        }
+
+        /**
+         * @return Request if need process primary fail response, {@code null} otherwise.
+         */
+        @Nullable GridNearAtomicAbstractUpdateRequest onPrimaryFail() {
+            if (finished())
+                return null;
+
+            /*
+             * When primary failed, even if primary response is received, it is possible it failed to send
+             * request to backup(s), need remap operation.
+             */
+            if (req.fullSync() && !req.nodeFailedResponse()) {
+                req.resetResponse();
+
+                return req;
+            }
+
+            return req.response() == null ? req : null;
         }
 
-        if (storeFuture() && !cctx.mvcc().addAtomicFuture(futVer, this))
-            return null;
+        /**
+         * @param nodeId Node ID.
+         * @param res Response.
+         * @return Request if need process primary response, {@code null} otherwise.
+         */
+        @Nullable GridNearAtomicAbstractUpdateRequest processPrimaryResponse(UUID nodeId, GridNearAtomicUpdateResponse res) {
+            assert req.nodeId().equals(nodeId);
 
-        return futVer;
+            if (res.nodeLeftResponse())
+                return onPrimaryFail();
+
+            if (finished())
+                return null;
+
+            return req.response() == null ? req : null;
+        }
+
+        /**
+         * @param nodeId Node ID.
+         * @return {@code True} if request processing finished.
+         */
+        DhtLeftResult onDhtNodeLeft(UUID nodeId) {
+            if (req.writeSynchronizationMode() != FULL_SYNC || dhtNodes == null || finished())
+                return DhtLeftResult.NOT_DONE;
+
+            if (dhtNodes.remove(nodeId) && dhtNodes.isEmpty()) {
+                if (hasRes)
+                    return DhtLeftResult.DONE;
+                else
+                    return !req.needPrimaryResponse() ? DhtLeftResult.ALL_RCVD_CHECK_PRIMARY : DhtLeftResult.NOT_DONE;
+            }
+
+            return DhtLeftResult.NOT_DONE;
+        }
+
+        /**
+         * @param nodeId Node ID.
+         * @param res Response.
+         * @return {@code True} if request processing finished.
+         */
+        boolean onDhtResponse(UUID nodeId, GridDhtAtomicNearResponse res) {
+            assert req.writeSynchronizationMode() == FULL_SYNC : req;
+
+            if (finished())
+                return false;
+
+            if (res.hasResult())
+                hasRes = true;
+
+            if (dhtNodes == null) {
+                if (rcvd == null)
+                    rcvd = new HashSet<>();
+
+                rcvd.add(nodeId);
+
+                return false;
+            }
+
+            return dhtNodes.remove(nodeId) && finished();
+        }
+
+        /**
+         * @param res Response.
+         * @param cctx Cache context.
+         * @return {@code True} if request processing finished.
+         */
+        boolean onPrimaryResponse(GridNearAtomicUpdateResponse res, GridCacheContext cctx) {
+            assert !finished() : this;
+
+            hasRes = true;
+
+            boolean onRes = req.onResponse(res);
+
+            assert onRes;
+
+            if (res.error() != null || res.remapTopologyVersion() != null) {
+                dhtNodes = Collections.emptySet(); // Mark as finished.
+
+                return true;
+            }
+
+            assert res.returnValue() != null : res;
+
+            if (res.dhtNodes() != null)
+                initDhtNodes(res.dhtNodes(), cctx);
+
+            return finished();
+        }
+
+        /**
+         * @param nodeIds Node IDs.
+         * @param cctx Context.
+         */
+        private void initDhtNodes(List<UUID> nodeIds, GridCacheContext cctx) {
+            assert dhtNodes == null || req.initMappingLocally();
+
+            Set<UUID> dhtNodes0 = dhtNodes;
+
+            dhtNodes = null;
+
+            for (UUID dhtNodeId : nodeIds) {
+                if (F.contains(rcvd, dhtNodeId))
+                    continue;
+
+                if (req.initMappingLocally() && !F.contains(dhtNodes0, dhtNodeId))
+                    continue;
+
+                if (cctx.discovery().node(dhtNodeId) != null) {
+                    if (dhtNodes == null)
+                        dhtNodes = U.newHashSet(nodeIds.size());
+
+                    dhtNodes.add(dhtNodeId);
+                }
+            }
+
+            if (dhtNodes == null)
+                dhtNodes = Collections.emptySet();
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(PrimaryRequestState.class, this,
+                "primary", primaryId(),
+                "needPrimaryRes", req.needPrimaryResponse(),
+                "primaryRes", req.response() != null,
+                "done", finished());
+        }
+    }
+
+    /**
+     *
+     */
+    enum DhtLeftResult {
+        /** All responses and operation result are received. */
+        DONE,
+
+        /** Not all responses are received. */
+        NOT_DONE,
+
+        /**
+         * All backups failed and response from primary is not required,
+         * in this case in FULL_SYNC mode need send additional request
+         * on primary to ensure FULL_SYNC guarantee.
+         */
+        ALL_RCVD_CHECK_PRIMARY
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridNearAtomicAbstractUpdateFuture.class, this, super.toString());
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java
index bee2ecd..a43bfb0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java
@@ -17,18 +17,28 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 
+import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.UUID;
 import javax.cache.expiry.ExpiryPolicy;
 import javax.cache.processor.EntryProcessor;
+import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheEntryPredicate;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheDeployable;
 import org.apache.ignite.internal.processors.cache.GridCacheMessage;
 import org.apache.ignite.internal.processors.cache.GridCacheOperation;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -38,106 +48,331 @@ public abstract class GridNearAtomicAbstractUpdateRequest extends GridCacheMessa
     /** Message index. */
     public static final int CACHE_MSG_IDX = nextIndexId();
 
+    /** . */
+    private static final int NEED_PRIMARY_RES_FLAG_MASK = 0x01;
+
+    /** Topology locked flag. Set if atomic update is performed inside TX or explicit lock. */
+    private static final int TOP_LOCKED_FLAG_MASK = 0x02;
+
+    /** Skip write-through to a persistent storage. */
+    private static final int SKIP_STORE_FLAG_MASK = 0x04;
+
+    /** Keep binary flag. */
+    private static final int KEEP_BINARY_FLAG_MASK = 0x08;
+
+    /** Return value flag. */
+    private static final int RET_VAL_FLAG_MASK = 0x10;
+
+    /** Target node ID. */
+    @GridDirectTransient
+    protected UUID nodeId;
+
+    /** Future version. */
+    protected long futId;
+
+    /** Topology version. */
+    protected AffinityTopologyVersion topVer;
+
+    /** Write synchronization mode. */
+    protected CacheWriteSynchronizationMode syncMode;
+
+    /** Update operation. */
+    protected GridCacheOperation op;
+
+    /** Subject ID. */
+    protected UUID subjId;
+
+    /** Task name hash. */
+    protected int taskNameHash;
+
+    /** Compressed boolean flags. Make sure 'toString' is updated when add new flag. */
+    @GridToStringExclude
+    protected byte flags;
+
+    /** */
+    @GridDirectTransient
+    private GridNearAtomicUpdateResponse res;
+
     /**
-     * @return Mapped node ID.
+     *
      */
-    public abstract UUID nodeId();
+    public GridNearAtomicAbstractUpdateRequest() {
+        // No-op.
+    }
 
     /**
+     * Constructor.
+     *
+     * @param cacheId Cache ID.
      * @param nodeId Node ID.
+     * @param futId Future ID.
+     * @param topVer Topology version.
+     * @param topLocked Topology locked flag.
+     * @param syncMode Synchronization mode.
+     * @param op Cache update operation.
+     * @param retval Return value required flag.
+     * @param subjId Subject ID.
+     * @param taskNameHash Task name hash code.
+     * @param needPrimaryRes {@code True} if near node waits for primary response.
+     * @param skipStore Skip write-through to a persistent storage.
+     * @param keepBinary Keep binary flag.
+     * @param addDepInfo Deployment info flag.
+     */
+    protected GridNearAtomicAbstractUpdateRequest(
+        int cacheId,
+        UUID nodeId,
+        long futId,
+        @NotNull AffinityTopologyVersion topVer,
+        boolean topLocked,
+        CacheWriteSynchronizationMode syncMode,
+        GridCacheOperation op,
+        boolean retval,
+        @Nullable UUID subjId,
+        int taskNameHash,
+        boolean needPrimaryRes,
+        boolean skipStore,
+        boolean keepBinary,
+        boolean addDepInfo
+    ) {
+        this.cacheId = cacheId;
+        this.nodeId = nodeId;
+        this.futId = futId;
+        this.topVer = topVer;
+        this.syncMode = syncMode;
+        this.op = op;
+        this.subjId = subjId;
+        this.taskNameHash = taskNameHash;
+        this.addDepInfo = addDepInfo;
+
+        if (needPrimaryRes)
+            needPrimaryResponse(true);
+        if (topLocked)
+            topologyLocked(true);
+        if (retval)
+            returnValue(true);
+        if (skipStore)
+            skipStore(true);
+        if (keepBinary)
+            keepBinary(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public final AffinityTopologyVersion topologyVersion() {
+        return topVer;
+    }
+
+    /** {@inheritDoc} */
+    @Override public final int lookupIndex() {
+        return CACHE_MSG_IDX;
+    }
+
+    /** {@inheritDoc} */
+    @Override public final boolean addDeploymentInfo() {
+        return addDepInfo;
+    }
+
+    /** {@inheritDoc} */
+    @Override public final IgniteLogger messageLogger(GridCacheSharedContext ctx) {
+        return ctx.atomicMessageLogger();
+    }
+
+    /**
+     * @return {@code True} if near node is able to initialize update mapping locally.
+     */
+    boolean initMappingLocally() {
+        return !needPrimaryResponse() && fullSync();
+    }
+
+    /**
+     * @return {@code True} if near node waits for primary response.
+     */
+    boolean needPrimaryResponse() {
+        return isFlag(NEED_PRIMARY_RES_FLAG_MASK);
+    }
+
+    /**
+     * @param needRes {@code True} if near node waits for primary response.
+     */
+    void needPrimaryResponse(boolean needRes) {
+        setFlag(needRes, NEED_PRIMARY_RES_FLAG_MASK);
+    }
+
+    /**
+     * @return {@code True} if update is processed in {@link CacheWriteSynchronizationMode#FULL_SYNC} mode.
+     */
+    boolean fullSync() {
+        assert syncMode != null;
+
+        return syncMode == CacheWriteSynchronizationMode.FULL_SYNC;
+    }
+
+    /**
+     * @return Task name hash code.
+     */
+    public int taskNameHash() {
+        return taskNameHash;
+    }
+
+    /**
+     * @return Update opreation.
      */
-    public abstract void nodeId(UUID nodeId);
+    public GridCacheOperation operation() {
+        return op;
+    }
 
     /**
      * @return Subject ID.
      */
-    public abstract UUID subjectId();
+    public UUID subjectId() {
+        return subjId;
+    }
 
     /**
-     * @return Task name hash.
+     * @return Target node ID.
      */
-    public abstract int taskNameHash();
+    public UUID nodeId() {
+        return nodeId;
+    }
 
     /**
-     * @return Future version.
+     * @return Near node future ID.
      */
-    public abstract GridCacheVersion futureVersion();
+    public long futureId() {
+        return futId;
+    }
 
     /**
-     * @return Flag indicating whether this is fast-map udpate.
+     * @return Write synchronization mode.
      */
-    public abstract boolean fastMap();
+    public final CacheWriteSynchronizationMode writeSynchronizationMode() {
+        return syncMode;
+    }
 
     /**
-     * @return Update version for fast-map request.
+     * @param res Response.
+     * @return {@code True} if current response was {@code null}.
      */
-    public abstract GridCacheVersion updateVersion();
+    public boolean onResponse(GridNearAtomicUpdateResponse res) {
+        if (this.res == null) {
+            this.res = res;
+
+            return true;
+        }
+
+        return false;
+    }
 
     /**
-     * @return Topology locked flag.
+     *
      */
-    public abstract boolean topologyLocked();
+    void resetResponse() {
+        this.res = null;
+    }
 
     /**
-     * @return {@code True} if request sent from client node.
+     * @return Response.
      */
-    public abstract boolean clientRequest();
+    @Nullable public GridNearAtomicUpdateResponse response() {
+        return res;
+    }
 
     /**
-     * @return Cache write synchronization mode.
+     * @return {@code True} if received notification about primary fail.
      */
-    public abstract CacheWriteSynchronizationMode writeSynchronizationMode();
+    boolean nodeFailedResponse() {
+        return res != null && res.nodeLeftResponse();
+    }
 
     /**
-     * @return Expiry policy.
+     * @return Topology locked flag.
      */
-    public abstract ExpiryPolicy expiry();
+    final boolean topologyLocked() {
+        return isFlag(TOP_LOCKED_FLAG_MASK);
+    }
+
+    /**
+     * @param val {@code True} if topology is locked on near node.
+     */
+    private void topologyLocked(boolean val) {
+        setFlag(val, TOP_LOCKED_FLAG_MASK);
+    }
 
     /**
      * @return Return value flag.
      */
-    public abstract boolean returnValue();
+    public final boolean returnValue() {
+        return isFlag(RET_VAL_FLAG_MASK);
+    }
 
     /**
-     * @return Filter.
+     * @param val Return value flag.
      */
-    @Nullable public abstract CacheEntryPredicate[] filter();
+    public final void returnValue(boolean val) {
+        setFlag(val, RET_VAL_FLAG_MASK);
+    }
 
     /**
      * @return Skip write-through to a persistent storage.
      */
-    public abstract boolean skipStore();
+    public final boolean skipStore() {
+        return isFlag(SKIP_STORE_FLAG_MASK);
+    }
+
+    /**
+     * @param val Skip store flag.
+     */
+    public void skipStore(boolean val) {
+        setFlag(val, SKIP_STORE_FLAG_MASK);
+    }
 
     /**
      * @return Keep binary flag.
      */
-    public abstract boolean keepBinary();
+    public final boolean keepBinary() {
+        return isFlag(KEEP_BINARY_FLAG_MASK);
+    }
 
     /**
-     * @return Update operation.
+     * @param val Keep binary flag.
      */
-    public abstract GridCacheOperation operation();
+    public void keepBinary(boolean val) {
+        setFlag(val, KEEP_BINARY_FLAG_MASK);
+    }
 
     /**
-     * @return Optional arguments for entry processor.
+     * Sets flag mask.
+     *
+     * @param flag Set or clear.
+     * @param mask Mask.
      */
-    @Nullable public abstract Object[] invokeArguments();
+    private void setFlag(boolean flag, int mask) {
+        flags = flag ? (byte)(flags | mask) : (byte)(flags & ~mask);
+    }
 
     /**
-     * @return Flag indicating whether this request contains primary keys.
+     * Reads flag mask.
+     *
+     * @param mask Mask to read.
+     * @return Flag value.
      */
-    public abstract boolean hasPrimary();
+    private boolean isFlag(int mask) {
+        return (flags & mask) != 0;
+    }
 
     /**
-     * @param res Response.
-     * @return {@code True} if current response was {@code null}.
+     * @return Expiry policy.
      */
-    public abstract boolean onResponse(GridNearAtomicUpdateResponse res);
+    public abstract ExpiryPolicy expiry();
 
     /**
-     * @return Response.
+     * @return Filter.
      */
-    @Nullable public abstract GridNearAtomicUpdateResponse response();
+    @Nullable public abstract CacheEntryPredicate[] filter();
+
+    /**
+     * @return Optional arguments for entry processor.
+     */
+    @Nullable public abstract Object[] invokeArguments();
 
     /**
      * @param key Key to add.
@@ -145,14 +380,12 @@ public abstract class GridNearAtomicAbstractUpdateRequest extends GridCacheMessa
      * @param conflictTtl Conflict TTL (optional).
      * @param conflictExpireTime Conflict expire time (optional).
      * @param conflictVer Conflict version (optional).
-     * @param primary If given key is primary on this mapping.
      */
-    public abstract void addUpdateEntry(KeyCacheObject key,
+    abstract void addUpdateEntry(KeyCacheObject key,
         @Nullable Object val,
         long conflictTtl,
         long conflictExpireTime,
-        @Nullable GridCacheVersion conflictVer,
-        boolean primary);
+        @Nullable GridCacheVersion conflictVer);
 
     /**
      * @return Keys for this update request.
@@ -182,7 +415,6 @@ public abstract class GridNearAtomicAbstractUpdateRequest extends GridCacheMessa
      */
     public abstract CacheObject writeValue(int idx);
 
-
     /**
      * @return Conflict versions.
      */
@@ -223,4 +455,170 @@ public abstract class GridNearAtomicAbstractUpdateRequest extends GridCacheMessa
      * @return Key.
      */
     public abstract KeyCacheObject key(int idx);
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 10;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 3:
+                if (!writer.writeByte("flags", flags))
+                    return false;
+
+                writer.incrementState();
+
+            case 4:
+                if (!writer.writeLong("futId", futId))
+                    return false;
+
+                writer.incrementState();
+
+            case 5:
+                if (!writer.writeByte("op", op != null ? (byte)op.ordinal() : -1))
+                    return false;
+
+                writer.incrementState();
+
+            case 6:
+                if (!writer.writeUuid("subjId", subjId))
+                    return false;
+
+                writer.incrementState();
+
+            case 7:
+                if (!writer.writeByte("syncMode", syncMode != null ? (byte)syncMode.ordinal() : -1))
+                    return false;
+
+                writer.incrementState();
+
+            case 8:
+                if (!writer.writeInt("taskNameHash", taskNameHash))
+                    return false;
+
+                writer.incrementState();
+
+            case 9:
+                if (!writer.writeMessage("topVer", topVer))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 3:
+                flags = reader.readByte("flags");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 4:
+                futId = reader.readLong("futId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 5:
+                byte opOrd;
+
+                opOrd = reader.readByte("op");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                op = GridCacheOperation.fromOrdinal(opOrd);
+
+                reader.incrementState();
+
+            case 6:
+                subjId = reader.readUuid("subjId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 7:
+                byte syncModeOrd;
+
+                syncModeOrd = reader.readByte("syncMode");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                syncMode = CacheWriteSynchronizationMode.fromOrdinal(syncModeOrd);
+
+                reader.incrementState();
+
+            case 8:
+                taskNameHash = reader.readInt("taskNameHash");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 9:
+                topVer = reader.readMessage("topVer");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(GridNearAtomicAbstractUpdateRequest.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        StringBuilder flags = new StringBuilder();
+
+        if (needPrimaryResponse())
+            appendFlag(flags, "needRes");
+        if (topologyLocked())
+            appendFlag(flags, "topLock");
+        if (skipStore())
+            appendFlag(flags, "skipStore");
+        if (keepBinary())
+            appendFlag(flags, "keepBinary");
+        if (returnValue())
+            appendFlag(flags, "retVal");
+
+        return S.toString(GridNearAtomicAbstractUpdateRequest.class, this,
+            "flags", flags.toString());
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicCheckUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicCheckUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicCheckUpdateRequest.java
new file mode 100644
index 0000000..4d0726a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicCheckUpdateRequest.java
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.processors.cache.GridCacheMessage;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ *
+ */
+public class GridNearAtomicCheckUpdateRequest extends GridCacheMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Cache message index. */
+    public static final int CACHE_MSG_IDX = nextIndexId();
+
+    /** */
+    @GridDirectTransient
+    private GridNearAtomicAbstractUpdateRequest updateReq;
+
+    /** */
+    private int partId;
+
+    /** */
+    private long futId;
+
+    /**
+     *
+     */
+    public GridNearAtomicCheckUpdateRequest() {
+        // No-op.
+    }
+
+    /**
+     * @param updateReq Related update request.
+     */
+    GridNearAtomicCheckUpdateRequest(GridNearAtomicAbstractUpdateRequest updateReq) {
+        assert updateReq != null && updateReq.fullSync() : updateReq;
+
+        this.updateReq = updateReq;
+        this.cacheId = updateReq.cacheId();
+        this.partId = updateReq.partition();
+        this.futId = updateReq.futureId();
+
+        assert partId >= 0;
+    }
+
+    /**
+     * @return Future ID on near node.
+     */
+    public final long futureId() {
+        return futId;
+    }
+
+    /**
+     * @return Related update request.
+     */
+    GridNearAtomicAbstractUpdateRequest updateRequest() {
+        return updateReq;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int partition() {
+        return partId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int lookupIndex() {
+        return CACHE_MSG_IDX;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean addDeploymentInfo() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -47;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 5;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 3:
+                if (!writer.writeLong("futId", futId))
+                    return false;
+
+                writer.incrementState();
+
+            case 4:
+                if (!writer.writeInt("partId", partId))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 3:
+                futId = reader.readLong("futId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 4:
+                partId = reader.readInt("partId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(GridNearAtomicCheckUpdateRequest.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridNearAtomicCheckUpdateRequest.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java
index 08c2474..c381333 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java
@@ -26,7 +26,6 @@ import java.util.UUID;
 import javax.cache.expiry.ExpiryPolicy;
 import javax.cache.processor.EntryProcessor;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.internal.GridDirectCollection;
 import org.apache.ignite.internal.GridDirectTransient;
@@ -41,6 +40,7 @@ import org.apache.ignite.internal.processors.cache.distributed.IgniteExternaliza
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.GridLongList;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -61,56 +61,6 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Target node ID. */
-    @GridDirectTransient
-    private UUID nodeId;
-
-    /** Future version. */
-    private GridCacheVersion futVer;
-
-    /** Update version. Set to non-null if fastMap is {@code true}. */
-    private GridCacheVersion updateVer;
-
-    /** Topology version. */
-    private AffinityTopologyVersion topVer;
-
-    /** Write synchronization mode. */
-    private CacheWriteSynchronizationMode syncMode;
-
-    /** Update operation. */
-    private GridCacheOperation op;
-
-    /** Subject ID. */
-    protected UUID subjId;
-
-    /** Task name hash. */
-    protected int taskNameHash;
-
-    /** */
-    @GridDirectTransient
-    private GridNearAtomicUpdateResponse res;
-
-    /** Fast map flag. */
-    protected boolean fastMap;
-
-    /** Topology locked flag. Set if atomic update is performed inside TX or explicit lock. */
-    protected boolean topLocked;
-
-    /** Flag indicating whether request contains primary keys. */
-    protected boolean hasPrimary;
-
-    /** Skip write-through to a persistent storage. */
-    protected boolean skipStore;
-
-    /** */
-    protected boolean clientReq;
-
-    /** Keep binary flag. */
-    protected boolean keepBinary;
-
-    /** Return value flag. */
-    protected boolean retval;
-
     /** Keys to update. */
     @GridToStringInclude
     @GridDirectCollection(KeyCacheObject.class)
@@ -120,10 +70,6 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
     @GridDirectCollection(CacheObject.class)
     private List<CacheObject> vals;
 
-    /** Partitions of keys. */
-    @GridDirectCollection(int.class)
-    private List<Integer> partIds;
-
     /** Entry processors. */
     @GridDirectTransient
     private List<EntryProcessor<Object, Object, Object>> entryProcessors;
@@ -175,9 +121,7 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
      *
      * @param cacheId Cache ID.
      * @param nodeId Node ID.
-     * @param futVer Future version.
-     * @param fastMap Fast map scheme flag.
-     * @param updateVer Update version set if fast map is performed.
+     * @param futId Future ID.
      * @param topVer Topology version.
      * @param topLocked Topology locked flag.
      * @param syncMode Synchronization mode.
@@ -190,16 +134,13 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
      * @param taskNameHash Task name hash code.
      * @param skipStore Skip write-through to a persistent storage.
      * @param keepBinary Keep binary flag.
-     * @param clientReq Client node request flag.
      * @param addDepInfo Deployment info flag.
      * @param maxEntryCnt Maximum entries count.
      */
     GridNearAtomicFullUpdateRequest(
         int cacheId,
         UUID nodeId,
-        GridCacheVersion futVer,
-        boolean fastMap,
-        @Nullable GridCacheVersion updateVer,
+        long futId,
         @NotNull AffinityTopologyVersion topVer,
         boolean topLocked,
         CacheWriteSynchronizationMode syncMode,
@@ -210,34 +151,29 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
         @Nullable CacheEntryPredicate[] filter,
         @Nullable UUID subjId,
         int taskNameHash,
+        boolean needPrimaryRes,
         boolean skipStore,
         boolean keepBinary,
-        boolean clientReq,
         boolean addDepInfo,
         int maxEntryCnt
     ) {
-        assert futVer != null;
-
-        this.cacheId = cacheId;
-        this.nodeId = nodeId;
-        this.futVer = futVer;
-        this.fastMap = fastMap;
-        this.updateVer = updateVer;
-
-        this.topVer = topVer;
-        this.topLocked = topLocked;
-        this.syncMode = syncMode;
-        this.op = op;
-        this.retval = retval;
+        super(cacheId,
+            nodeId,
+            futId,
+            topVer,
+            topLocked,
+            syncMode,
+            op,
+            retval,
+            subjId,
+            taskNameHash,
+            needPrimaryRes,
+            skipStore,
+            keepBinary,
+            addDepInfo);
         this.expiryPlc = expiryPlc;
         this.invokeArgs = invokeArgs;
         this.filter = filter;
-        this.subjId = subjId;
-        this.taskNameHash = taskNameHash;
-        this.skipStore = skipStore;
-        this.keepBinary = keepBinary;
-        this.clientReq = clientReq;
-        this.addDepInfo = addDepInfo;
 
         // By default ArrayList expands to array of 10 elements on first add. We cannot guess how many entries
         // will be added to request because of unknown affinity distribution. However, we DO KNOW how many keys
@@ -246,84 +182,6 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
         initSize = Math.min(maxEntryCnt, 10);
 
         keys = new ArrayList<>(initSize);
-
-        partIds = new ArrayList<>(initSize);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int lookupIndex() {
-        return CACHE_MSG_IDX;
-    }
-
-    /** {@inheritDoc} */
-    @Override public UUID nodeId() {
-        return nodeId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void nodeId(UUID nodeId) {
-        this.nodeId = nodeId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public UUID subjectId() {
-        return subjId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int taskNameHash() {
-        return taskNameHash;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridCacheVersion futureVersion() {
-        return futVer;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridCacheVersion updateVersion() {
-        return updateVer;
-    }
-
-    /** {@inheritDoc} */
-    @Override public AffinityTopologyVersion topologyVersion() {
-        return topVer;
-    }
-
-    /** {@inheritDoc} */
-    @Override public CacheWriteSynchronizationMode writeSynchronizationMode() {
-        return syncMode;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridCacheOperation operation() {
-        return op;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean onResponse(GridNearAtomicUpdateResponse res) {
-        if (this.res == null) {
-            this.res = res;
-
-            return true;
-        }
-
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override @Nullable public GridNearAtomicUpdateResponse response() {
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean addDeploymentInfo() {
-        return addDepInfo;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteLogger messageLogger(GridCacheSharedContext ctx) {
-        return ctx.atomicMessageLogger();
     }
 
     /** {@inheritDoc} */
@@ -331,8 +189,7 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
         @Nullable Object val,
         long conflictTtl,
         long conflictExpireTime,
-        @Nullable GridCacheVersion conflictVer,
-        boolean primary) {
+        @Nullable GridCacheVersion conflictVer) {
         EntryProcessor<Object, Object, Object> entryProcessor = null;
 
         if (op == TRANSFORM) {
@@ -344,7 +201,6 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
         assert val != null || op == DELETE;
 
         keys.add(key);
-        partIds.add(key.partition());
 
         if (entryProcessor != null) {
             if (entryProcessors == null)
@@ -361,8 +217,6 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
             vals.add((CacheObject)val);
         }
 
-        hasPrimary |= primary;
-
         // In case there is no conflict, do not create the list.
         if (conflictVer != null) {
             if (conflictVers == null) {
@@ -407,6 +261,8 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
 
     /** {@inheritDoc} */
     @Override public int size() {
+        assert keys != null;
+
         return keys != null ? keys.size() : 0;
     }
 
@@ -488,41 +344,6 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
     }
 
     /** {@inheritDoc} */
-    @Override public boolean fastMap() {
-        return fastMap;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean topologyLocked() {
-        return topLocked;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean clientRequest() {
-        return clientReq;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean returnValue() {
-        return retval;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean skipStore() {
-        return skipStore;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean keepBinary() {
-        return keepBinary;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean hasPrimary() {
-        return hasPrimary;
-    }
-
-    /** {@inheritDoc} */
     @Override @Nullable public CacheEntryPredicate[] filter() {
         return filter;
     }
@@ -600,18 +421,13 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
         }
         else
             finishUnmarshalCacheObjects(vals, cctx, ldr);
-
-        if (partIds != null && !partIds.isEmpty()) {
-            assert partIds.size() == keys.size();
-
-            for (int i = 0; i < keys.size(); i++)
-                keys.get(i).partition(partIds.get(i));
-        }
     }
 
     /** {@inheritDoc} */
     @Override public int partition() {
-        return partIds != null && !partIds.isEmpty() ? partIds.get(0) : -1;
+        assert !F.isEmpty(keys);
+
+        return keys.get(0).partition();
     }
 
     /** {@inheritDoc} */
@@ -629,145 +445,55 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
         }
 
         switch (writer.state()) {
-            case 3:
-                if (!writer.writeBoolean("clientReq", clientReq))
-                    return false;
-
-                writer.incrementState();
-
-            case 4:
-                if (!writer.writeMessage("conflictExpireTimes", conflictExpireTimes))
-                    return false;
-
-                writer.incrementState();
-
-            case 5:
-                if (!writer.writeMessage("conflictTtls", conflictTtls))
-                    return false;
-
-                writer.incrementState();
-
-            case 6:
-                if (!writer.writeCollection("conflictVers", conflictVers, MessageCollectionItemType.MSG))
-                    return false;
-
-                writer.incrementState();
-
-            case 7:
-                if (!writer.writeCollection("entryProcessorsBytes", entryProcessorsBytes, MessageCollectionItemType.BYTE_ARR))
-                    return false;
-
-                writer.incrementState();
-
-            case 8:
-                if (!writer.writeByteArray("expiryPlcBytes", expiryPlcBytes))
-                    return false;
-
-                writer.incrementState();
-
-            case 9:
-                if (!writer.writeBoolean("fastMap", fastMap))
-                    return false;
-
-                writer.incrementState();
-
             case 10:
-                if (!writer.writeObjectArray("filter", filter, MessageCollectionItemType.MSG))
+                if (!writer.writeMessage("conflictExpireTimes", conflictExpireTimes))
                     return false;
 
                 writer.incrementState();
 
             case 11:
-                if (!writer.writeMessage("futVer", futVer))
+                if (!writer.writeMessage("conflictTtls", conflictTtls))
                     return false;
 
                 writer.incrementState();
 
             case 12:
-                if (!writer.writeBoolean("hasPrimary", hasPrimary))
+                if (!writer.writeCollection("conflictVers", conflictVers, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
             case 13:
-                if (!writer.writeObjectArray("invokeArgsBytes", invokeArgsBytes, MessageCollectionItemType.BYTE_ARR))
+                if (!writer.writeCollection("entryProcessorsBytes", entryProcessorsBytes, MessageCollectionItemType.BYTE_ARR))
                     return false;
 
                 writer.incrementState();
 
             case 14:
-                if (!writer.writeBoolean("keepBinary", keepBinary))
+                if (!writer.writeByteArray("expiryPlcBytes", expiryPlcBytes))
                     return false;
 
                 writer.incrementState();
 
             case 15:
-                if (!writer.writeCollection("keys", keys, MessageCollectionItemType.MSG))
+                if (!writer.writeObjectArray("filter", filter, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
             case 16:
-                if (!writer.writeByte("op", op != null ? (byte)op.ordinal() : -1))
+                if (!writer.writeObjectArray("invokeArgsBytes", invokeArgsBytes, MessageCollectionItemType.BYTE_ARR))
                     return false;
 
                 writer.incrementState();
 
             case 17:
-                if (!writer.writeCollection("partIds", partIds, MessageCollectionItemType.INT))
+                if (!writer.writeCollection("keys", keys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
             case 18:
-                if (!writer.writeBoolean("retval", retval))
-                    return false;
-
-                writer.incrementState();
-
-            case 19:
-                if (!writer.writeBoolean("skipStore", skipStore))
-                    return false;
-
-                writer.incrementState();
-
-            case 20:
-                if (!writer.writeUuid("subjId", subjId))
-                    return false;
-
-                writer.incrementState();
-
-            case 21:
-                if (!writer.writeByte("syncMode", syncMode != null ? (byte)syncMode.ordinal() : -1))
-                    return false;
-
-                writer.incrementState();
-
-            case 22:
-                if (!writer.writeInt("taskNameHash", taskNameHash))
-                    return false;
-
-                writer.incrementState();
-
-            case 23:
-                if (!writer.writeBoolean("topLocked", topLocked))
-                    return false;
-
-                writer.incrementState();
-
-            case 24:
-                if (!writer.writeMessage("topVer", topVer))
-                    return false;
-
-                writer.incrementState();
-
-            case 25:
-                if (!writer.writeMessage("updateVer", updateVer))
-                    return false;
-
-                writer.incrementState();
-
-            case 26:
                 if (!writer.writeCollection("vals", vals, MessageCollectionItemType.MSG))
                     return false;
 
@@ -789,64 +515,8 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
             return false;
 
         switch (reader.state()) {
-            case 3:
-                clientReq = reader.readBoolean("clientReq");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 4:
-                conflictExpireTimes = reader.readMessage("conflictExpireTimes");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 5:
-                conflictTtls = reader.readMessage("conflictTtls");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 6:
-                conflictVers = reader.readCollection("conflictVers", MessageCollectionItemType.MSG);
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 7:
-                entryProcessorsBytes = reader.readCollection("entryProcessorsBytes", MessageCollectionItemType.BYTE_ARR);
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 8:
-                expiryPlcBytes = reader.readByteArray("expiryPlcBytes");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 9:
-                fastMap = reader.readBoolean("fastMap");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
             case 10:
-                filter = reader.readObjectArray("filter", MessageCollectionItemType.MSG, CacheEntryPredicate.class);
+                conflictExpireTimes = reader.readMessage("conflictExpireTimes");
 
                 if (!reader.isLastRead())
                     return false;
@@ -854,7 +524,7 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
                 reader.incrementState();
 
             case 11:
-                futVer = reader.readMessage("futVer");
+                conflictTtls = reader.readMessage("conflictTtls");
 
                 if (!reader.isLastRead())
                     return false;
@@ -862,7 +532,7 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
                 reader.incrementState();
 
             case 12:
-                hasPrimary = reader.readBoolean("hasPrimary");
+                conflictVers = reader.readCollection("conflictVers", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
                     return false;
@@ -870,7 +540,7 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
                 reader.incrementState();
 
             case 13:
-                invokeArgsBytes = reader.readObjectArray("invokeArgsBytes", MessageCollectionItemType.BYTE_ARR, byte[].class);
+                entryProcessorsBytes = reader.readCollection("entryProcessorsBytes", MessageCollectionItemType.BYTE_ARR);
 
                 if (!reader.isLastRead())
                     return false;
@@ -878,7 +548,7 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
                 reader.incrementState();
 
             case 14:
-                keepBinary = reader.readBoolean("keepBinary");
+                expiryPlcBytes = reader.readByteArray("expiryPlcBytes");
 
                 if (!reader.isLastRead())
                     return false;
@@ -886,7 +556,7 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
                 reader.incrementState();
 
             case 15:
-                keys = reader.readCollection("keys", MessageCollectionItemType.MSG);
+                filter = reader.readObjectArray("filter", MessageCollectionItemType.MSG, CacheEntryPredicate.class);
 
                 if (!reader.isLastRead())
                     return false;
@@ -894,19 +564,15 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
                 reader.incrementState();
 
             case 16:
-                byte opOrd;
-
-                opOrd = reader.readByte("op");
+                invokeArgsBytes = reader.readObjectArray("invokeArgsBytes", MessageCollectionItemType.BYTE_ARR, byte[].class);
 
                 if (!reader.isLastRead())
                     return false;
 
-                op = GridCacheOperation.fromOrdinal(opOrd);
-
                 reader.incrementState();
 
             case 17:
-                partIds = reader.readCollection("partIds", MessageCollectionItemType.INT);
+                keys = reader.readCollection("keys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
                     return false;
@@ -914,74 +580,6 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
                 reader.incrementState();
 
             case 18:
-                retval = reader.readBoolean("retval");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 19:
-                skipStore = reader.readBoolean("skipStore");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 20:
-                subjId = reader.readUuid("subjId");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 21:
-                byte syncModeOrd;
-
-                syncModeOrd = reader.readByte("syncMode");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                syncMode = CacheWriteSynchronizationMode.fromOrdinal(syncModeOrd);
-
-                reader.incrementState();
-
-            case 22:
-                taskNameHash = reader.readInt("taskNameHash");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 23:
-                topLocked = reader.readBoolean("topLocked");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 24:
-                topVer = reader.readMessage("topVer");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 25:
-                updateVer = reader.readMessage("updateVer");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 26:
                 vals = reader.readCollection("vals", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -1013,12 +611,13 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 27;
+        return 19;
     }
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return S.toString(GridNearAtomicFullUpdateRequest.class, this, "filter", Arrays.toString(filter),
+        return S.toString(GridNearAtomicFullUpdateRequest.class, this,
+            "filter", Arrays.toString(filter),
             "parent", super.toString());
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFilterRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFilterRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFilterRequest.java
index e0c24b2..78582b0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFilterRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFilterRequest.java
@@ -58,9 +58,7 @@ public class GridNearAtomicSingleUpdateFilterRequest extends GridNearAtomicSingl
      *
      * @param cacheId Cache ID.
      * @param nodeId Node ID.
-     * @param futVer Future version.
-     * @param fastMap Fast map scheme flag.
-     * @param updateVer Update version set if fast map is performed.
+     * @param futId Future ID.
      * @param topVer Topology version.
      * @param topLocked Topology locked flag.
      * @param syncMode Synchronization mode.
@@ -71,15 +69,12 @@ public class GridNearAtomicSingleUpdateFilterRequest extends GridNearAtomicSingl
      * @param taskNameHash Task name hash code.
      * @param skipStore Skip write-through to a persistent storage.
      * @param keepBinary Keep binary flag.
-     * @param clientReq Client node request flag.
      * @param addDepInfo Deployment info flag.
      */
     GridNearAtomicSingleUpdateFilterRequest(
         int cacheId,
         UUID nodeId,
-        GridCacheVersion futVer,
-        boolean fastMap,
-        @Nullable GridCacheVersion updateVer,
+        long futId,
         @NotNull AffinityTopologyVersion topVer,
         boolean topLocked,
         CacheWriteSynchronizationMode syncMode,
@@ -88,17 +83,15 @@ public class GridNearAtomicSingleUpdateFilterRequest extends GridNearAtomicSingl
         @Nullable CacheEntryPredicate[] filter,
         @Nullable UUID subjId,
         int taskNameHash,
+        boolean needPrimaryRes,
         boolean skipStore,
         boolean keepBinary,
-        boolean clientReq,
         boolean addDepInfo
     ) {
         super(
             cacheId,
             nodeId,
-            futVer,
-            fastMap,
-            updateVer,
+            futId,
             topVer,
             topLocked,
             syncMode,
@@ -106,9 +99,9 @@ public class GridNearAtomicSingleUpdateFilterRequest extends GridNearAtomicSingl
             retval,
             subjId,
             taskNameHash,
+            needPrimaryRes,
             skipStore,
             keepBinary,
-            clientReq,
             addDepInfo
         );
 
@@ -173,7 +166,7 @@ public class GridNearAtomicSingleUpdateFilterRequest extends GridNearAtomicSingl
         }
 
         switch (writer.state()) {
-            case 14:
+            case 12:
                 if (!writer.writeObjectArray("filter", filter, MessageCollectionItemType.MSG))
                     return false;
 
@@ -195,7 +188,7 @@ public class GridNearAtomicSingleUpdateFilterRequest extends GridNearAtomicSingl
             return false;
 
         switch (reader.state()) {
-            case 14:
+            case 12:
                 filter = reader.readObjectArray("filter", MessageCollectionItemType.MSG, CacheEntryPredicate.class);
 
                 if (!reader.isLastRead())
@@ -215,7 +208,7 @@ public class GridNearAtomicSingleUpdateFilterRequest extends GridNearAtomicSingl
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 15;
+        return 13;
     }
 
     /** {@inheritDoc} */


[6/8] ignite git commit: ignite-4705 Atomic cache protocol change: notify client node from backups

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicDeferredUpdateResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicDeferredUpdateResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicDeferredUpdateResponse.java
index 923b220..9fe183f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicDeferredUpdateResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicDeferredUpdateResponse.java
@@ -19,17 +19,18 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 
 import java.io.Externalizable;
 import java.nio.ByteBuffer;
-import java.util.Collection;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.internal.GridDirectCollection;
+import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.processors.cache.GridCacheDeployable;
 import org.apache.ignite.internal.processors.cache.GridCacheMessage;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
+import org.apache.ignite.internal.util.GridLongList;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Deferred dht atomic update response.
@@ -42,13 +43,12 @@ public class GridDhtAtomicDeferredUpdateResponse extends GridCacheMessage implem
     public static final int CACHE_MSG_IDX = nextIndexId();
 
     /** ACK future versions. */
-    @GridDirectCollection(GridCacheVersion.class)
-    private Collection<GridCacheVersion> futVers;
+    private GridLongList futIds;
 
-    /** {@inheritDoc} */
-    @Override public int lookupIndex() {
-        return CACHE_MSG_IDX;
-    }
+    /** */
+    @GridDirectTransient
+    @GridToStringExclude
+    private GridTimeoutObject timeoutSnd;
 
     /**
      * Empty constructor required by {@link Externalizable}
@@ -61,27 +61,42 @@ public class GridDhtAtomicDeferredUpdateResponse extends GridCacheMessage implem
      * Constructor.
      *
      * @param cacheId Cache ID.
-     * @param futVers Future versions.
-     * @param addDepInfo Deployment info.
+     * @param futIds Future IDs.
      */
-    public GridDhtAtomicDeferredUpdateResponse(int cacheId, Collection<GridCacheVersion> futVers, boolean addDepInfo) {
-        assert !F.isEmpty(futVers);
-
+    public GridDhtAtomicDeferredUpdateResponse(int cacheId, GridLongList futIds) {
         this.cacheId = cacheId;
-        this.futVers = futVers;
-        this.addDepInfo = addDepInfo;
+        this.futIds = futIds;
+    }
+
+    /**
+     * @param timeoutSnd Callback sending response on timeout.
+     */
+    void timeoutSender(@Nullable GridTimeoutObject timeoutSnd) {
+        this.timeoutSnd = timeoutSnd;
+    }
+
+    /**
+     * @return Callback sending response on timeout.
+     */
+    @Nullable GridTimeoutObject timeoutSender() {
+        return timeoutSnd;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int lookupIndex() {
+        return CACHE_MSG_IDX;
     }
 
     /** {@inheritDoc} */
     @Override public boolean addDeploymentInfo() {
-        return addDepInfo;
+        return false;
     }
 
     /**
-     * @return List of ACKed future versions.
+     * @return List of ACKed future ids.
      */
-    public Collection<GridCacheVersion> futureVersions() {
-        return futVers;
+    GridLongList futureIds() {
+        return futIds;
     }
 
     /** {@inheritDoc} */
@@ -105,7 +120,7 @@ public class GridDhtAtomicDeferredUpdateResponse extends GridCacheMessage implem
 
         switch (writer.state()) {
             case 3:
-                if (!writer.writeCollection("futVers", futVers, MessageCollectionItemType.MSG))
+                if (!writer.writeMessage("futIds", futIds))
                     return false;
 
                 writer.incrementState();
@@ -127,7 +142,7 @@ public class GridDhtAtomicDeferredUpdateResponse extends GridCacheMessage implem
 
         switch (reader.state()) {
             case 3:
-                futVers = reader.readCollection("futVers", MessageCollectionItemType.MSG);
+                futIds = reader.readMessage("futIds");
 
                 if (!reader.isLastRead())
                     return false;
@@ -148,4 +163,9 @@ public class GridDhtAtomicDeferredUpdateResponse extends GridCacheMessage implem
     @Override public byte fieldsCount() {
         return 4;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridDhtAtomicDeferredUpdateResponse.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicNearResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicNearResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicNearResponse.java
new file mode 100644
index 0000000..08a7e28
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicNearResponse.java
@@ -0,0 +1,314 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.GridCacheMessage;
+import org.apache.ignite.internal.processors.cache.GridCacheReturn;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateRequest.DHT_ATOMIC_HAS_RESULT_MASK;
+
+/**
+ * Message sent from DHT nodes to near node in FULL_SYNC mode.
+ */
+public class GridDhtAtomicNearResponse extends GridCacheMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Message index. */
+    public static final int CACHE_MSG_IDX = nextIndexId();
+
+    /** */
+    private int partId;
+
+    /** */
+    private long futId;
+
+    /** */
+    private UUID primaryId;
+
+    /** */
+    @GridToStringExclude
+    private byte flags;
+
+    /** */
+    @GridToStringInclude
+    private UpdateErrors errs;
+
+    /**
+     *
+     */
+    public GridDhtAtomicNearResponse() {
+        // No-op.
+    }
+
+    /**
+     * @param cacheId Cache ID.
+     * @param partId Partition.
+     * @param futId Future ID.
+     * @param primaryId Primary node ID.
+     * @param flags Flags.
+     */
+    public GridDhtAtomicNearResponse(int cacheId,
+        int partId,
+        long futId,
+        UUID primaryId,
+        byte flags)
+    {
+        assert primaryId != null;
+
+        this.cacheId = cacheId;
+        this.partId = partId;
+        this.futId = futId;
+        this.primaryId = primaryId;
+        this.flags = flags;
+    }
+
+    /**
+     * @return Errors.
+     */
+    @Nullable UpdateErrors errors() {
+        return errs;
+    }
+
+    /**
+     * @param errs Errors.
+     */
+    public void errors(UpdateErrors errs) {
+        this.errs = errs;
+    }
+
+    /**
+     * @return Primary node ID.
+     */
+    UUID primaryId() {
+        return primaryId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int partition() {
+        return partId;
+    }
+
+    /**
+     * @param key Key.
+     * @param e Error.
+     */
+    public void addFailedKey(KeyCacheObject key, Throwable e) {
+        if (errs == null)
+            errs = new UpdateErrors();
+
+        errs.addFailedKey(key, e);
+    }
+
+    /**
+     * @return Operation result.
+     */
+    public GridCacheReturn result() {
+        assert hasResult() : this;
+
+        return new GridCacheReturn(true, true);
+    }
+
+    /**
+     * @return {@code True} if response contains operation result.
+     */
+    boolean hasResult() {
+        return isFlag(DHT_ATOMIC_HAS_RESULT_MASK);
+    }
+
+    /**
+     * Reads flag mask.
+     *
+     * @param mask Mask to read.
+     * @return Flag value.
+     */
+    private boolean isFlag(int mask) {
+        return (flags & mask) != 0;
+    }
+
+    /**
+     * @return Future ID.
+     */
+    public long futureId() {
+        return futId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int lookupIndex() {
+        return CACHE_MSG_IDX;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -45;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 8;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean addDeploymentInfo() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException {
+        super.prepareMarshal(ctx);
+
+        if (errs != null)
+            errs.prepareMarshal(this, ctx.cacheContext(cacheId));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void finishUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException {
+        super.finishUnmarshal(ctx, ldr);
+
+        if (errs != null)
+            errs.finishUnmarshal(this, ctx.cacheContext(cacheId), ldr);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 3:
+                if (!writer.writeMessage("errs", errs))
+                    return false;
+
+                writer.incrementState();
+
+            case 4:
+                if (!writer.writeByte("flags", flags))
+                    return false;
+
+                writer.incrementState();
+
+            case 5:
+                if (!writer.writeLong("futId", futId))
+                    return false;
+
+                writer.incrementState();
+
+            case 6:
+                if (!writer.writeInt("partId", partId))
+                    return false;
+
+                writer.incrementState();
+
+            case 7:
+                if (!writer.writeUuid("primaryId", primaryId))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 3:
+                errs = reader.readMessage("errs");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 4:
+                flags = reader.readByte("flags");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 5:
+                futId = reader.readLong("futId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 6:
+                partId = reader.readInt("partId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 7:
+                primaryId = reader.readUuid("primaryId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(GridDhtAtomicNearResponse.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        StringBuilder flags = new StringBuilder();
+
+        if (hasResult())
+            appendFlag(flags, "hasRes");
+
+        return S.toString(GridDhtAtomicNearResponse.class, this,
+            "flags", flags.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java
index 0dc2754..8ebe9c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java
@@ -24,16 +24,11 @@ import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.util.typedef.CI2;
-import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteProductVersion;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
@@ -45,64 +40,45 @@ class GridDhtAtomicSingleUpdateFuture extends GridDhtAtomicAbstractUpdateFuture
     private static final long serialVersionUID = 0L;
 
     /** */
-    private static final IgniteProductVersion SINGLE_UPDATE_REQUEST = IgniteProductVersion.fromString("1.7.4");
-
-    /** Future keys. */
-    private KeyCacheObject key;
-
-    /** Entries with readers. */
-    private GridDhtCacheEntry nearReaderEntry;
+    private boolean allUpdated;
 
     /**
      * @param cctx Cache context.
-     * @param completionCb Callback to invoke when future is completed.
      * @param writeVer Write version.
      * @param updateReq Update request.
-     * @param updateRes Update response.
      */
     GridDhtAtomicSingleUpdateFuture(
         GridCacheContext cctx,
-        CI2<GridNearAtomicAbstractUpdateRequest, GridNearAtomicUpdateResponse> completionCb,
         GridCacheVersion writeVer,
-        GridNearAtomicAbstractUpdateRequest updateReq,
-        GridNearAtomicUpdateResponse updateRes
+        GridNearAtomicAbstractUpdateRequest updateReq
     ) {
-        super(cctx,
-            completionCb,
-            writeVer,
-            updateReq,
-            updateRes);
+        super(cctx, writeVer, updateReq);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean sendAllToDht() {
+        return allUpdated;
     }
 
     /** {@inheritDoc} */
     @Override protected void addDhtKey(KeyCacheObject key, List<ClusterNode> dhtNodes) {
-        assert this.key == null || this.key.equals(key) : this.key;
+        if (mappings == null) {
+            allUpdated = true;
 
-        if (mappings == null)
             mappings = U.newHashMap(dhtNodes.size());
-
-        this.key = key;
+        }
     }
 
     /** {@inheritDoc} */
     @Override protected void addNearKey(KeyCacheObject key, Collection<UUID> readers) {
-        assert this.key == null || this.key.equals(key) : this.key;
-
         if (mappings == null)
             mappings = U.newHashMap(readers.size());
-
-        this.key = key;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void addNearReaderEntry(GridDhtCacheEntry entry) {
-        nearReaderEntry = entry;
     }
 
     /** {@inheritDoc} */
     @Override protected GridDhtAtomicAbstractUpdateRequest createRequest(
-        ClusterNode node,
-        GridCacheVersion futVer,
+        UUID nodeId,
+        long futId,
         GridCacheVersion writeVer,
         CacheWriteSynchronizationMode syncMode,
         @NotNull AffinityTopologyVersion topVer,
@@ -110,11 +86,11 @@ class GridDhtAtomicSingleUpdateFuture extends GridDhtAtomicAbstractUpdateFuture
         long conflictExpireTime,
         @Nullable GridCacheVersion conflictVer
     ) {
-        if (canUseSingleRequest(node, ttl, conflictExpireTime, conflictVer)) {
+        if (canUseSingleRequest(ttl, conflictExpireTime, conflictVer)) {
             return new GridDhtAtomicSingleUpdateRequest(
                 cctx.cacheId(),
-                node.id(),
-                futVer,
+                nodeId,
+                futId,
                 writeVer,
                 syncMode,
                 topVer,
@@ -127,68 +103,37 @@ class GridDhtAtomicSingleUpdateFuture extends GridDhtAtomicAbstractUpdateFuture
         else {
             return new GridDhtAtomicUpdateRequest(
                 cctx.cacheId(),
-                node.id(),
-                futVer,
+                nodeId,
+                futId,
                 writeVer,
                 syncMode,
                 topVer,
-                false,
                 updateReq.subjectId(),
                 updateReq.taskNameHash(),
                 null,
                 cctx.deploymentEnabled(),
                 updateReq.keepBinary(),
-                updateReq.skipStore());
+                updateReq.skipStore(),
+                false);
         }
     }
 
-    /** {@inheritDoc} */
-    @Override public void onResult(UUID nodeId, GridDhtAtomicUpdateResponse updateRes) {
-        if (log.isDebugEnabled())
-            log.debug("Received DHT atomic update future result [nodeId=" + nodeId + ", updateRes=" + updateRes + ']');
-
-        if (updateRes.error() != null)
-            this.updateRes.addFailedKeys(updateRes.failedKeys(), updateRes.error());
-
-        if (!F.isEmpty(updateRes.nearEvicted())) {
-            try {
-                assert nearReaderEntry != null;
-
-                nearReaderEntry.removeReader(nodeId, updateRes.messageId());
-            }
-            catch (GridCacheEntryRemovedException e) {
-                if (log.isDebugEnabled())
-                    log.debug("Entry with evicted reader was removed [entry=" + nearReaderEntry + ", err=" + e + ']');
-            }
-        }
-
-        registerResponse(nodeId);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void addFailedKeys(GridNearAtomicUpdateResponse updateRes, Throwable err) {
-        updateRes.addFailedKey(key, err);
-    }
-
     /**
-     * @param node Target node
      * @param ttl TTL.
      * @param conflictExpireTime Conflict expire time.
      * @param conflictVer Conflict version.
      * @return {@code True} if it is possible to use {@link GridDhtAtomicSingleUpdateRequest}.
      */
-    private boolean canUseSingleRequest(ClusterNode node,
-        long ttl,
+    private boolean canUseSingleRequest(long ttl,
         long conflictExpireTime,
         @Nullable GridCacheVersion conflictVer) {
-        return node.version().compareToIgnoreTimestamp(SINGLE_UPDATE_REQUEST) >= 0 &&
-            (ttl == CU.TTL_NOT_CHANGED) &&
+        return (ttl == CU.TTL_NOT_CHANGED) &&
             (conflictExpireTime == CU.EXPIRE_TIME_CALCULATE) &&
             conflictVer == null;
     }
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return S.toString(GridDhtAtomicSingleUpdateFuture.class, this);
+        return S.toString(GridDhtAtomicSingleUpdateFuture.class, this, "super", super.toString());
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java
index a7e6c24..6b92c02 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java
@@ -38,9 +38,6 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.internal.processors.cache.GridCacheUtils.KEEP_BINARY_FLAG_MASK;
-import static org.apache.ignite.internal.processors.cache.GridCacheUtils.SKIP_STORE_FLAG_MASK;
-
 /**
  *
  */
@@ -48,30 +45,6 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Near cache key flag. */
-    private static final int NEAR_FLAG_MASK = 0x80;
-
-    /** Future version. */
-    protected GridCacheVersion futVer;
-
-    /** Write version. */
-    protected GridCacheVersion writeVer;
-
-    /** Write synchronization mode. */
-    protected CacheWriteSynchronizationMode syncMode;
-
-    /** Topology version. */
-    protected AffinityTopologyVersion topVer;
-
-    /** Subject ID. */
-    protected UUID subjId;
-
-    /** Task name hash. */
-    protected int taskNameHash;
-
-    /** Additional flags. */
-    protected byte flags;
-
     /** Key to update. */
     @GridToStringInclude
     protected KeyCacheObject key;
@@ -87,9 +60,6 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
     /** Partition. */
     protected long updateCntr;
 
-    /** */
-    protected int partId;
-
     /**
      * Empty constructor required by {@link Externalizable}.
      */
@@ -102,7 +72,7 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
      *
      * @param cacheId Cache ID.
      * @param nodeId Node ID.
-     * @param futVer Future version.
+     * @param futId Future ID.
      * @param writeVer Write version for cache values.
      * @param syncMode Cache write synchronization mode.
      * @param topVer Topology version.
@@ -115,7 +85,7 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
     GridDhtAtomicSingleUpdateRequest(
         int cacheId,
         UUID nodeId,
-        GridCacheVersion futVer,
+        long futId,
         GridCacheVersion writeVer,
         CacheWriteSynchronizationMode syncMode,
         @NotNull AffinityTopologyVersion topVer,
@@ -125,19 +95,17 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
         boolean keepBinary,
         boolean skipStore
     ) {
-        super(cacheId, nodeId);
-        this.futVer = futVer;
-        this.writeVer = writeVer;
-        this.syncMode = syncMode;
-        this.topVer = topVer;
-        this.subjId = subjId;
-        this.taskNameHash = taskNameHash;
-        this.addDepInfo = addDepInfo;
-
-        if (skipStore)
-            setFlag(true, SKIP_STORE_FLAG_MASK);
-        if (keepBinary)
-            setFlag(true, KEEP_BINARY_FLAG_MASK);
+        super(cacheId,
+            nodeId,
+            futId,
+            writeVer,
+            syncMode,
+            topVer,
+            subjId,
+            taskNameHash,
+            addDepInfo,
+            keepBinary,
+            skipStore);
     }
 
     /**
@@ -148,7 +116,6 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
      * @param conflictExpireTime Conflict expire time (optional).
      * @param conflictVer Conflict version (optional).
      * @param addPrevVal If {@code true} adds previous value.
-     * @param partId Partition.
      * @param prevVal Previous value.
      * @param updateCntr Update counter.
      */
@@ -159,7 +126,6 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
         long conflictExpireTime,
         @Nullable GridCacheVersion conflictVer,
         boolean addPrevVal,
-        int partId,
         @Nullable CacheObject prevVal,
         long updateCntr
     ) {
@@ -167,11 +133,11 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
         assert ttl <= 0 : ttl;
         assert conflictExpireTime <= 0 : conflictExpireTime;
         assert conflictVer == null : conflictVer;
+        assert key.partition() >= 0 : key;
 
         near(false);
 
         this.key = key;
-        this.partId = partId;
         this.val = val;
 
         if (addPrevVal)
@@ -194,6 +160,7 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
         long expireTime) {
         assert entryProcessor == null;
         assert ttl <= 0 : ttl;
+        assert key.partition() >= 0 : key;
 
         near(true);
 
@@ -222,11 +189,6 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
     }
 
     /** {@inheritDoc} */
-    @Override public boolean skipStore() {
-        return isFlag(SKIP_STORE_FLAG_MASK);
-    }
-
-    /** {@inheritDoc} */
     @Override public KeyCacheObject key(int idx) {
         assert idx == 0 : idx;
 
@@ -235,14 +197,11 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
 
     /** {@inheritDoc} */
     @Override public int partition() {
-        return partId;
-    }
+        int p = key.partition();
 
-    /** {@inheritDoc} */
-    @Override public int partitionId(int idx) {
-        assert idx == 0 : idx;
+        assert p >= 0;
 
-        return partId;
+        return p;
     }
 
     /** {@inheritDoc} */
@@ -267,31 +226,6 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
     }
 
     /** {@inheritDoc} */
-    @Override public GridCacheVersion futureVersion() {
-        return futVer;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridCacheVersion writeVersion() {
-        return writeVer;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int taskNameHash() {
-        return taskNameHash;
-    }
-
-    /** {@inheritDoc} */
-    @Override public UUID subjectId() {
-        return subjId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public CacheWriteSynchronizationMode writeSynchronizationMode() {
-        return syncMode;
-    }
-
-    /** {@inheritDoc} */
     @Override @Nullable public CacheObject previousValue(int idx) {
         assert idx == 0 : idx;
 
@@ -360,25 +294,6 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
     }
 
     /** {@inheritDoc} */
-    @Override public boolean keepBinary() {
-        return isFlag(KEEP_BINARY_FLAG_MASK);
-    }
-
-    /**
-     *
-     */
-    private boolean near() {
-        return isFlag(NEAR_FLAG_MASK);
-    }
-
-    /**
-     *
-     */
-    private void near(boolean near) {
-        setFlag(near, NEAR_FLAG_MASK);
-    }
-
-    /** {@inheritDoc} */
     @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException {
         super.prepareMarshal(ctx);
 
@@ -403,8 +318,6 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
         finishUnmarshalObject(val, cctx, ldr);
 
         finishUnmarshalObject(prevVal, cctx, ldr);
-
-        key.partition(partId);
     }
 
     /** {@inheritDoc} */
@@ -422,78 +335,30 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
         }
 
         switch (writer.state()) {
-            case 3:
-                if (!writer.writeByte("flags", flags))
-                    return false;
-
-                writer.incrementState();
-
-            case 4:
-                if (!writer.writeMessage("futVer", futVer))
-                    return false;
-
-                writer.incrementState();
-
-            case 5:
+            case 12:
                 if (!writer.writeMessage("key", key))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
-                if (!writer.writeInt("partId", partId))
-                    return false;
-
-                writer.incrementState();
-
-            case 7:
+            case 13:
                 if (!writer.writeMessage("prevVal", prevVal))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
-                if (!writer.writeUuid("subjId", subjId))
-                    return false;
-
-                writer.incrementState();
-
-            case 9:
-                if (!writer.writeByte("syncMode", syncMode != null ? (byte)syncMode.ordinal() : -1))
-                    return false;
-
-                writer.incrementState();
-
-            case 10:
-                if (!writer.writeInt("taskNameHash", taskNameHash))
-                    return false;
-
-                writer.incrementState();
-
-            case 11:
-                if (!writer.writeMessage("topVer", topVer))
-                    return false;
-
-                writer.incrementState();
-
-            case 12:
+            case 14:
                 if (!writer.writeLong("updateCntr", updateCntr))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
+            case 15:
                 if (!writer.writeMessage("val", val))
                     return false;
 
                 writer.incrementState();
 
-            case 14:
-                if (!writer.writeMessage("writeVer", writeVer))
-                    return false;
-
-                writer.incrementState();
-
         }
 
         return true;
@@ -510,23 +375,7 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
             return false;
 
         switch (reader.state()) {
-            case 3:
-                flags = reader.readByte("flags");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 4:
-                futVer = reader.readMessage("futVer");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 5:
+            case 12:
                 key = reader.readMessage("key");
 
                 if (!reader.isLastRead())
@@ -534,15 +383,7 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
 
                 reader.incrementState();
 
-            case 6:
-                partId = reader.readInt("partId");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 7:
+            case 13:
                 prevVal = reader.readMessage("prevVal");
 
                 if (!reader.isLastRead())
@@ -550,43 +391,7 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
 
                 reader.incrementState();
 
-            case 8:
-                subjId = reader.readUuid("subjId");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 9:
-                byte syncModeOrd;
-
-                syncModeOrd = reader.readByte("syncMode");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                syncMode = CacheWriteSynchronizationMode.fromOrdinal(syncModeOrd);
-
-                reader.incrementState();
-
-            case 10:
-                taskNameHash = reader.readInt("taskNameHash");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 11:
-                topVer = reader.readMessage("topVer");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 12:
+            case 14:
                 updateCntr = reader.readLong("updateCntr");
 
                 if (!reader.isLastRead())
@@ -594,7 +399,7 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
 
                 reader.incrementState();
 
-            case 13:
+            case 15:
                 val = reader.readMessage("val");
 
                 if (!reader.isLastRead())
@@ -602,14 +407,6 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
 
                 reader.incrementState();
 
-            case 14:
-                writeVer = reader.readMessage("writeVer");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
         }
 
         return reader.afterMessageRead(GridDhtAtomicSingleUpdateRequest.class);
@@ -652,27 +449,7 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 15;
-    }
-
-    /**
-     * Sets flag mask.
-     *
-     * @param flag Set or clear.
-     * @param mask Mask.
-     */
-    private void setFlag(boolean flag, int mask) {
-        flags = flag ? (byte)(flags | mask) : (byte)(flags & ~mask);
-    }
-
-    /**
-     * Reags flag mask.
-     *
-     * @param mask Mask to read.
-     * @return Flag value.
-     */
-    private boolean isFlag(int mask) {
-        return (flags & mask) != 0;
+        return 16;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
index 5429adc..5d5ddf0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
@@ -17,22 +17,15 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 
-import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.UUID;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.util.typedef.CI2;
-import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.NotNull;
@@ -45,89 +38,45 @@ class GridDhtAtomicUpdateFuture extends GridDhtAtomicAbstractUpdateFuture {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Future keys. */
-    private final Collection<KeyCacheObject> keys;
-
-    /** Entries with readers. */
-    private Map<KeyCacheObject, GridDhtCacheEntry> nearReadersEntries;
-
+    /** */
+    private int updateCntr;
 
     /**
      * @param cctx Cache context.
-     * @param completionCb Callback to invoke when future is completed.
      * @param writeVer Write version.
      * @param updateReq Update request.
-     * @param updateRes Update response.
      */
     GridDhtAtomicUpdateFuture(
         GridCacheContext cctx,
-        CI2<GridNearAtomicAbstractUpdateRequest, GridNearAtomicUpdateResponse> completionCb,
         GridCacheVersion writeVer,
-        GridNearAtomicAbstractUpdateRequest updateReq,
-        GridNearAtomicUpdateResponse updateRes
+        GridNearAtomicAbstractUpdateRequest updateReq
     ) {
-        super(cctx,
-            completionCb,
-            writeVer,
-            updateReq,
-            updateRes);
+        super(cctx, writeVer, updateReq);
 
-        keys = new ArrayList<>(updateReq.size());
         mappings = U.newHashMap(updateReq.size());
     }
 
     /** {@inheritDoc} */
-    @Override protected void addDhtKey(KeyCacheObject key, List<ClusterNode> dhtNodes) {
-        keys.add(key);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void addNearKey(KeyCacheObject key, Collection<UUID> readers) {
-        keys.add(key);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void addNearReaderEntry(GridDhtCacheEntry entry) {
-        if (nearReadersEntries == null)
-            nearReadersEntries = new HashMap<>();
-
-        nearReadersEntries.put(entry.key(), entry);
+    @Override protected boolean sendAllToDht() {
+        return updateCntr == updateReq.size();
     }
 
     /** {@inheritDoc} */
-    @Override public void onResult(UUID nodeId, GridDhtAtomicUpdateResponse updateRes) {
-        if (log.isDebugEnabled())
-            log.debug("Received DHT atomic update future result [nodeId=" + nodeId + ", updateRes=" + updateRes + ']');
-
-        if (updateRes.error() != null)
-            this.updateRes.addFailedKeys(updateRes.failedKeys(), updateRes.error());
-
-        if (!F.isEmpty(updateRes.nearEvicted())) {
-            for (KeyCacheObject key : updateRes.nearEvicted()) {
-                GridDhtCacheEntry entry = nearReadersEntries.get(key);
-
-                try {
-                    entry.removeReader(nodeId, updateRes.messageId());
-                }
-                catch (GridCacheEntryRemovedException e) {
-                    if (log.isDebugEnabled())
-                        log.debug("Entry with evicted reader was removed [entry=" + entry + ", err=" + e + ']');
-                }
-            }
-        }
+    @Override protected void addDhtKey(KeyCacheObject key, List<ClusterNode> dhtNodes) {
+        assert updateCntr < updateReq.size();
 
-        registerResponse(nodeId);
+        updateCntr++;
     }
 
     /** {@inheritDoc} */
-    @Override protected void addFailedKeys(GridNearAtomicUpdateResponse updateRes, Throwable err) {
-        for (KeyCacheObject key : keys)
-            updateRes.addFailedKey(key, err);
+    @Override protected void addNearKey(KeyCacheObject key, Collection<UUID> readers) {
+        // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override protected GridDhtAtomicAbstractUpdateRequest createRequest(ClusterNode node,
-        GridCacheVersion futVer,
+    @Override protected GridDhtAtomicAbstractUpdateRequest createRequest(
+        UUID nodeId,
+        long futId,
         GridCacheVersion writeVer,
         CacheWriteSynchronizationMode syncMode,
         @NotNull AffinityTopologyVersion topVer,
@@ -137,22 +86,22 @@ class GridDhtAtomicUpdateFuture extends GridDhtAtomicAbstractUpdateFuture {
     ) {
         return new GridDhtAtomicUpdateRequest(
             cctx.cacheId(),
-            node.id(),
-            futVer,
+            nodeId,
+            futId,
             writeVer,
             syncMode,
             topVer,
-            false,
             updateReq.subjectId(),
             updateReq.taskNameHash(),
             null,
             cctx.deploymentEnabled(),
             updateReq.keepBinary(),
-            updateReq.skipStore());
+            updateReq.skipStore(),
+            false);
     }
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return S.toString(GridDhtAtomicUpdateFuture.class, this);
+        return S.toString(GridDhtAtomicUpdateFuture.class, this, "super", super.toString());
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
index 7144963..6b8af8d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
@@ -44,8 +44,6 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.internal.processors.cache.GridCacheUtils.SKIP_STORE_FLAG_MASK;
-
 /**
  * Lite dht cache backup update request.
  */
@@ -53,15 +51,6 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Future version. */
-    private GridCacheVersion futVer;
-
-    /** Write version. */
-    private GridCacheVersion writeVer;
-
-    /** Topology version. */
-    private AffinityTopologyVersion topVer;
-
     /** Keys to update. */
     @GridToStringInclude
     @GridDirectCollection(KeyCacheObject.class)
@@ -93,9 +82,6 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
     /** Near expire times. */
     private GridLongList nearExpireTimes;
 
-    /** Write synchronization mode. */
-    private CacheWriteSynchronizationMode syncMode;
-
     /** Near cache keys to update. */
     @GridToStringInclude
     @GridDirectCollection(KeyCacheObject.class)
@@ -132,25 +118,9 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
     /** Entry processor arguments bytes. */
     private byte[][] invokeArgsBytes;
 
-    /** Subject ID. */
-    private UUID subjId;
-
-    /** Task name hash. */
-    private int taskNameHash;
-
     /** Partition. */
     private GridLongList updateCntrs;
 
-    /** */
-    @GridDirectTransient
-    private List<Integer> partIds;
-
-    /** Keep binary flag. */
-    private boolean keepBinary;
-
-    /** Additional flags. */
-    private byte flags;
-
     /**
      * Empty constructor required by {@link Externalizable}.
      */
@@ -163,7 +133,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
      *
      * @param cacheId Cache ID.
      * @param nodeId Node ID.
-     * @param futVer Future version.
+     * @param futId Future ID.
      * @param writeVer Write version for cache values.
      * @param invokeArgs Optional arguments for entry processor.
      * @param syncMode Cache write synchronization mode.
@@ -176,38 +146,36 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
     public GridDhtAtomicUpdateRequest(
         int cacheId,
         UUID nodeId,
-        GridCacheVersion futVer,
+        long futId,
         GridCacheVersion writeVer,
         CacheWriteSynchronizationMode syncMode,
         @NotNull AffinityTopologyVersion topVer,
-        boolean forceTransformBackups,
         UUID subjId,
         int taskNameHash,
         Object[] invokeArgs,
         boolean addDepInfo,
         boolean keepBinary,
-        boolean skipStore
+        boolean skipStore,
+        boolean forceTransformBackups
     ) {
-        super(cacheId, nodeId);
-
-        this.futVer = futVer;
-        this.writeVer = writeVer;
-        this.syncMode = syncMode;
-        this.topVer = topVer;
-        this.forceTransformBackups = forceTransformBackups;
-        this.subjId = subjId;
-        this.taskNameHash = taskNameHash;
+        super(cacheId,
+            nodeId,
+            futId,
+            writeVer,
+            syncMode,
+            topVer,
+            subjId,
+            taskNameHash,
+            addDepInfo,
+            keepBinary,
+            skipStore);
 
         assert invokeArgs == null || forceTransformBackups;
 
+        this.forceTransformBackups = forceTransformBackups;
         this.invokeArgs = invokeArgs;
-        this.addDepInfo = addDepInfo;
-        this.keepBinary = keepBinary;
-
-        setFlag(skipStore, SKIP_STORE_FLAG_MASK);
 
         keys = new ArrayList<>();
-        partIds = new ArrayList<>();
 
         if (forceTransformBackups) {
             entryProcessors = new ArrayList<>();
@@ -225,13 +193,12 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
         long conflictExpireTime,
         @Nullable GridCacheVersion conflictVer,
         boolean addPrevVal,
-        int partId,
         @Nullable CacheObject prevVal,
         long updateCntr
     ) {
-        keys.add(key);
+        assert key.partition() >= 0 : key;
 
-        partIds.add(partId);
+        keys.add(key);
 
         if (forceTransformBackups) {
             assert entryProcessor != null;
@@ -298,6 +265,8 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
         EntryProcessor<Object, Object, Object> entryProcessor,
         long ttl,
         long expireTime) {
+        assert key.partition() >= 0 : key;
+
         if (nearKeys == null) {
             nearKeys = new ArrayList<>();
 
@@ -350,31 +319,6 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
     }
 
     /** {@inheritDoc} */
-    @Override public UUID subjectId() {
-        return subjId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int taskNameHash() {
-        return taskNameHash;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridCacheVersion futureVersion() {
-        return futVer;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridCacheVersion writeVersion() {
-        return writeVer;
-    }
-
-    /** {@inheritDoc} */
-    @Override public CacheWriteSynchronizationMode writeSynchronizationMode() {
-        return syncMode;
-    }
-
-    /** {@inheritDoc} */
     @Override public AffinityTopologyVersion topologyVersion() {
         return topVer;
     }
@@ -400,11 +344,6 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
     }
 
     /** {@inheritDoc} */
-    @Override public int partitionId(int idx) {
-        return partIds.get(idx);
-    }
-
-    /** {@inheritDoc} */
     @Override public Long updateCounter(int updCntr) {
         if (updateCntrs != null && updCntr < updateCntrs.size())
             return updateCntrs.get(updCntr);
@@ -486,7 +425,13 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
     /** {@inheritDoc} */
     @Override public int partition() {
-        return partIds != null && !partIds.isEmpty() ? partIds.get(0) : -1;
+        assert !F.isEmpty(keys) || !F.isEmpty(nearKeys);
+
+        int p = keys.size() > 0 ? keys.get(0).partition() : nearKeys.get(0).partition();
+
+        assert p >= 0;
+
+        return p;
     }
 
     /** {@inheritDoc} */
@@ -512,16 +457,6 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
     }
 
     /** {@inheritDoc} */
-    @Override public boolean keepBinary() {
-        return keepBinary;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean skipStore() {
-        return isFlag(SKIP_STORE_FLAG_MASK);
-    }
-
-    /** {@inheritDoc} */
     @Override @Nullable public Object[] invokeArguments() {
         return invokeArgs;
     }
@@ -584,13 +519,6 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
             if (nearEntryProcessors == null)
                 nearEntryProcessors = unmarshalCollection(nearEntryProcessorsBytes, ctx, ldr);
         }
-
-        if (partIds != null && !partIds.isEmpty()) {
-            assert partIds.size() == keys.size();
-
-            for (int i = 0; i < keys.size(); i++)
-                keys.get(i).partition(partIds.get(i));
-        }
     }
 
     /** {@inheritDoc} */
@@ -608,144 +536,96 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
         }
 
         switch (writer.state()) {
-            case 3:
+            case 12:
                 if (!writer.writeMessage("conflictExpireTimes", conflictExpireTimes))
                     return false;
 
                 writer.incrementState();
 
-            case 4:
+            case 13:
                 if (!writer.writeCollection("conflictVers", conflictVers, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 5:
+            case 14:
                 if (!writer.writeCollection("entryProcessorsBytes", entryProcessorsBytes, MessageCollectionItemType.BYTE_ARR))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
-                if (!writer.writeByte("flags", flags))
-                    return false;
-
-                writer.incrementState();
-
-            case 7:
+            case 15:
                 if (!writer.writeBoolean("forceTransformBackups", forceTransformBackups))
                     return false;
 
                 writer.incrementState();
 
-            case 8:
-                if (!writer.writeMessage("futVer", futVer))
-                    return false;
-
-                writer.incrementState();
-
-            case 9:
+            case 16:
                 if (!writer.writeObjectArray("invokeArgsBytes", invokeArgsBytes, MessageCollectionItemType.BYTE_ARR))
                     return false;
 
                 writer.incrementState();
 
-            case 10:
-                if (!writer.writeBoolean("keepBinary", keepBinary))
-                    return false;
-
-                writer.incrementState();
-
-            case 11:
+            case 17:
                 if (!writer.writeCollection("keys", keys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 12:
+            case 18:
                 if (!writer.writeCollection("nearEntryProcessorsBytes", nearEntryProcessorsBytes, MessageCollectionItemType.BYTE_ARR))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
+            case 19:
                 if (!writer.writeMessage("nearExpireTimes", nearExpireTimes))
                     return false;
 
                 writer.incrementState();
 
-            case 14:
+            case 20:
                 if (!writer.writeCollection("nearKeys", nearKeys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 15:
+            case 21:
                 if (!writer.writeMessage("nearTtls", nearTtls))
                     return false;
 
                 writer.incrementState();
 
-            case 16:
+            case 22:
                 if (!writer.writeCollection("nearVals", nearVals, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 17:
+            case 23:
                 if (!writer.writeCollection("prevVals", prevVals, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 18:
-                if (!writer.writeUuid("subjId", subjId))
-                    return false;
-
-                writer.incrementState();
-
-            case 19:
-                if (!writer.writeByte("syncMode", syncMode != null ? (byte)syncMode.ordinal() : -1))
-                    return false;
-
-                writer.incrementState();
-
-            case 20:
-                if (!writer.writeInt("taskNameHash", taskNameHash))
-                    return false;
-
-                writer.incrementState();
-
-            case 21:
-                if (!writer.writeMessage("topVer", topVer))
-                    return false;
-
-                writer.incrementState();
-
-            case 22:
+            case 24:
                 if (!writer.writeMessage("ttls", ttls))
                     return false;
 
                 writer.incrementState();
 
-            case 23:
+            case 25:
                 if (!writer.writeMessage("updateCntrs", updateCntrs))
                     return false;
 
                 writer.incrementState();
 
-            case 24:
+            case 26:
                 if (!writer.writeCollection("vals", vals, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 25:
-                if (!writer.writeMessage("writeVer", writeVer))
-                    return false;
-
-                writer.incrementState();
-
         }
 
         return true;
@@ -762,7 +642,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
             return false;
 
         switch (reader.state()) {
-            case 3:
+            case 12:
                 conflictExpireTimes = reader.readMessage("conflictExpireTimes");
 
                 if (!reader.isLastRead())
@@ -770,7 +650,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 4:
+            case 13:
                 conflictVers = reader.readCollection("conflictVers", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -778,7 +658,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 5:
+            case 14:
                 entryProcessorsBytes = reader.readCollection("entryProcessorsBytes", MessageCollectionItemType.BYTE_ARR);
 
                 if (!reader.isLastRead())
@@ -786,15 +666,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 6:
-                flags = reader.readByte("flags");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 7:
+            case 15:
                 forceTransformBackups = reader.readBoolean("forceTransformBackups");
 
                 if (!reader.isLastRead())
@@ -802,15 +674,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 8:
-                futVer = reader.readMessage("futVer");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 9:
+            case 16:
                 invokeArgsBytes = reader.readObjectArray("invokeArgsBytes", MessageCollectionItemType.BYTE_ARR, byte[].class);
 
                 if (!reader.isLastRead())
@@ -818,15 +682,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 10:
-                keepBinary = reader.readBoolean("keepBinary");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 11:
+            case 17:
                 keys = reader.readCollection("keys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -834,7 +690,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 12:
+            case 18:
                 nearEntryProcessorsBytes = reader.readCollection("nearEntryProcessorsBytes", MessageCollectionItemType.BYTE_ARR);
 
                 if (!reader.isLastRead())
@@ -842,7 +698,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 13:
+            case 19:
                 nearExpireTimes = reader.readMessage("nearExpireTimes");
 
                 if (!reader.isLastRead())
@@ -850,7 +706,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 14:
+            case 20:
                 nearKeys = reader.readCollection("nearKeys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -858,7 +714,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 15:
+            case 21:
                 nearTtls = reader.readMessage("nearTtls");
 
                 if (!reader.isLastRead())
@@ -866,7 +722,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 16:
+            case 22:
                 nearVals = reader.readCollection("nearVals", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -874,7 +730,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 17:
+            case 23:
                 prevVals = reader.readCollection("prevVals", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -882,43 +738,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 18:
-                subjId = reader.readUuid("subjId");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 19:
-                byte syncModeOrd;
-
-                syncModeOrd = reader.readByte("syncMode");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                syncMode = CacheWriteSynchronizationMode.fromOrdinal(syncModeOrd);
-
-                reader.incrementState();
-
-            case 20:
-                taskNameHash = reader.readInt("taskNameHash");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 21:
-                topVer = reader.readMessage("topVer");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 22:
+            case 24:
                 ttls = reader.readMessage("ttls");
 
                 if (!reader.isLastRead())
@@ -926,7 +746,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 23:
+            case 25:
                 updateCntrs = reader.readMessage("updateCntrs");
 
                 if (!reader.isLastRead())
@@ -934,7 +754,7 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 24:
+            case 26:
                 vals = reader.readCollection("vals", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -942,14 +762,6 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
                 reader.incrementState();
 
-            case 25:
-                writeVer = reader.readMessage("writeVer");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
         }
 
         return reader.afterMessageRead(GridDhtAtomicUpdateRequest.class);
@@ -968,30 +780,9 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 26;
-    }
-
-    /**
-     * Sets flag mask.
-     *
-     * @param flag Set or clear.
-     * @param mask Mask.
-     */
-    private void setFlag(boolean flag, int mask) {
-        flags = flag ? (byte)(flags | mask) : (byte)(flags & ~mask);
+        return 27;
     }
 
-    /**
-     * Reags flag mask.
-     *
-     * @param mask Mask to read.
-     * @return Flag value.
-     */
-    private boolean isFlag(int mask) {
-        return (flags & mask) != 0;
-    }
-
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridDhtAtomicUpdateRequest.class, this, "super", super.toString());

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java
index c3d3ca9..ab7aa6c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java
@@ -25,16 +25,13 @@ import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.GridDirectCollection;
-import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheDeployable;
 import org.apache.ignite.internal.processors.cache.GridCacheMessage;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
@@ -50,19 +47,10 @@ public class GridDhtAtomicUpdateResponse extends GridCacheMessage implements Gri
     public static final int CACHE_MSG_IDX = nextIndexId();
 
     /** Future version. */
-    private GridCacheVersion futVer;
+    private long futId;
 
-    /** Failed keys. */
-    @GridToStringInclude
-    @GridDirectCollection(KeyCacheObject.class)
-    private List<KeyCacheObject> failedKeys;
-
-    /** Update error. */
-    @GridDirectTransient
-    private IgniteCheckedException err;
-
-    /** Serialized update error. */
-    private byte[] errBytes;
+    /** */
+    private UpdateErrors errs;
 
     /** Evicted readers. */
     @GridToStringInclude
@@ -70,7 +58,7 @@ public class GridDhtAtomicUpdateResponse extends GridCacheMessage implements Gri
     private List<KeyCacheObject> nearEvicted;
 
     /** */
-    private int partId = -1;
+    private int partId;
 
     /**
      * Empty constructor required by {@link Externalizable}.
@@ -81,12 +69,14 @@ public class GridDhtAtomicUpdateResponse extends GridCacheMessage implements Gri
 
     /**
      * @param cacheId Cache ID.
-     * @param futVer Future version.
+     * @param partId Partition.
+     * @param futId Future ID.
      * @param addDepInfo Deployment info.
      */
-    public GridDhtAtomicUpdateResponse(int cacheId, GridCacheVersion futVer, boolean addDepInfo) {
+    public GridDhtAtomicUpdateResponse(int cacheId, int partId, long futId, boolean addDepInfo) {
         this.cacheId = cacheId;
-        this.futVer = futVer;
+        this.partId = partId;
+        this.futId = futId;
         this.addDepInfo = addDepInfo;
     }
 
@@ -98,8 +88,8 @@ public class GridDhtAtomicUpdateResponse extends GridCacheMessage implements Gri
     /**
      * @return Future version.
      */
-    public GridCacheVersion futureVersion() {
-        return futVer;
+    public long futureId() {
+        return futId;
     }
 
     /**
@@ -108,63 +98,29 @@ public class GridDhtAtomicUpdateResponse extends GridCacheMessage implements Gri
      * @param err Error.
      */
     public void onError(IgniteCheckedException err){
-        this.err = err;
+        if (errs == null)
+            errs = new UpdateErrors();
+
+        errs.onError(err);
     }
 
     /** {@inheritDoc} */
     @Override public IgniteCheckedException error() {
-        return err;
-    }
-
-    /**
-     * @return Failed keys.
-     */
-    public Collection<KeyCacheObject> failedKeys() {
-        return failedKeys;
-    }
-
-    /**
-     * Adds key to collection of failed keys.
-     *
-     * @param key Key to add.
-     * @param e Error cause.
-     */
-    public void addFailedKey(KeyCacheObject key, Throwable e) {
-        if (failedKeys == null)
-            failedKeys = new ArrayList<>();
-
-        failedKeys.add(key);
-
-        if (err == null)
-            err = new IgniteCheckedException("Failed to update keys on primary node.");
-
-        err.addSuppressed(e);
+        return errs != null ? errs.error() : null;
     }
 
     /**
      * @return Evicted readers.
      */
-    public Collection<KeyCacheObject> nearEvicted() {
+    Collection<KeyCacheObject> nearEvicted() {
         return nearEvicted;
     }
 
     /**
-     * Adds near evicted key..
-     *
-     * @param key Evicted key.
-     */
-    public void addNearEvicted(KeyCacheObject key) {
-        if (nearEvicted == null)
-            nearEvicted = new ArrayList<>();
-
-        nearEvicted.add(key);
-    }
-
-    /**
-     * @param partId Partition ID to set.
+     * @param nearEvicted Evicted near cache keys.
      */
-    public void partition(int partId) {
-        this.partId = partId;
+    void nearEvicted(List<KeyCacheObject> nearEvicted) {
+        this.nearEvicted = nearEvicted;
     }
 
     /** {@inheritDoc} */
@@ -178,12 +134,10 @@ public class GridDhtAtomicUpdateResponse extends GridCacheMessage implements Gri
 
         GridCacheContext cctx = ctx.cacheContext(cacheId);
 
-        prepareMarshalCacheObjects(failedKeys, cctx);
-
         prepareMarshalCacheObjects(nearEvicted, cctx);
 
-        if (errBytes == null)
-            errBytes = U.marshal(ctx, err);
+        if (errs != null)
+            errs.prepareMarshal(this, cctx);
     }
 
     /** {@inheritDoc} */
@@ -192,12 +146,10 @@ public class GridDhtAtomicUpdateResponse extends GridCacheMessage implements Gri
 
         GridCacheContext cctx = ctx.cacheContext(cacheId);
 
-        finishUnmarshalCacheObjects(failedKeys, cctx, ldr);
-
         finishUnmarshalCacheObjects(nearEvicted, cctx, ldr);
 
-        if (errBytes != null && err == null)
-            err = U.unmarshal(ctx, errBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
+        if (errs != null)
+            errs.finishUnmarshal(this, cctx, ldr);
     }
 
     /** {@inheritDoc} */
@@ -226,30 +178,24 @@ public class GridDhtAtomicUpdateResponse extends GridCacheMessage implements Gri
 
         switch (writer.state()) {
             case 3:
-                if (!writer.writeByteArray("errBytes", errBytes))
+                if (!writer.writeMessage("errs", errs))
                     return false;
 
                 writer.incrementState();
 
             case 4:
-                if (!writer.writeCollection("failedKeys", failedKeys, MessageCollectionItemType.MSG))
+                if (!writer.writeLong("futId", futId))
                     return false;
 
                 writer.incrementState();
 
             case 5:
-                if (!writer.writeMessage("futVer", futVer))
-                    return false;
-
-                writer.incrementState();
-
-            case 6:
                 if (!writer.writeCollection("nearEvicted", nearEvicted, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 6:
                 if (!writer.writeInt("partId", partId))
                     return false;
 
@@ -272,7 +218,7 @@ public class GridDhtAtomicUpdateResponse extends GridCacheMessage implements Gri
 
         switch (reader.state()) {
             case 3:
-                errBytes = reader.readByteArray("errBytes");
+                errs = reader.readMessage("errs");
 
                 if (!reader.isLastRead())
                     return false;
@@ -280,7 +226,7 @@ public class GridDhtAtomicUpdateResponse extends GridCacheMessage implements Gri
                 reader.incrementState();
 
             case 4:
-                failedKeys = reader.readCollection("failedKeys", MessageCollectionItemType.MSG);
+                futId = reader.readLong("futId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -288,14 +234,6 @@ public class GridDhtAtomicUpdateResponse extends GridCacheMessage implements Gri
                 reader.incrementState();
 
             case 5:
-                futVer = reader.readMessage("futVer");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 6:
                 nearEvicted = reader.readCollection("nearEvicted", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -303,7 +241,7 @@ public class GridDhtAtomicUpdateResponse extends GridCacheMessage implements Gri
 
                 reader.incrementState();
 
-            case 7:
+            case 6:
                 partId = reader.readInt("partId");
 
                 if (!reader.isLastRead())
@@ -323,7 +261,7 @@ public class GridDhtAtomicUpdateResponse extends GridCacheMessage implements Gri
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 8;
+        return 7;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractSingleUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractSingleUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractSingleUpdateRequest.java
index 61deeee..6811236 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractSingleUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractSingleUpdateRequest.java
@@ -18,19 +18,13 @@
 package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 
 import java.io.Externalizable;
-import java.nio.ByteBuffer;
 import java.util.UUID;
 import javax.cache.expiry.ExpiryPolicy;
-import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheEntryPredicate;
 import org.apache.ignite.internal.processors.cache.GridCacheOperation;
-import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.plugin.extensions.communication.MessageReader;
-import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
@@ -44,59 +38,6 @@ public abstract class GridNearAtomicAbstractSingleUpdateRequest extends GridNear
     /** */
     private static final CacheEntryPredicate[] NO_FILTER = new CacheEntryPredicate[0];
 
-    /** Fast map flag mask. */
-    private static final int FAST_MAP_FLAG_MASK = 0x1;
-
-    /** Flag indicating whether request contains primary keys. */
-    private static final int HAS_PRIMARY_FLAG_MASK = 0x2;
-
-    /** Topology locked flag. Set if atomic update is performed inside TX or explicit lock. */
-    private static final int TOP_LOCKED_FLAG_MASK = 0x4;
-
-    /** Skip write-through to a persistent storage. */
-    private static final int SKIP_STORE_FLAG_MASK = 0x8;
-
-    /** */
-    private static final int CLIENT_REQ_FLAG_MASK = 0x10;
-
-    /** Keep binary flag. */
-    private static final int KEEP_BINARY_FLAG_MASK = 0x20;
-
-    /** Return value flag. */
-    private static final int RET_VAL_FLAG_MASK = 0x40;
-
-    /** Target node ID. */
-    @GridDirectTransient
-    protected UUID nodeId;
-
-    /** Future version. */
-    protected GridCacheVersion futVer;
-
-    /** Update version. Set to non-null if fastMap is {@code true}. */
-    private GridCacheVersion updateVer;
-
-    /** Topology version. */
-    protected AffinityTopologyVersion topVer;
-
-    /** Write synchronization mode. */
-    protected CacheWriteSynchronizationMode syncMode;
-
-    /** Update operation. */
-    protected GridCacheOperation op;
-
-    /** Subject ID. */
-    protected UUID subjId;
-
-    /** Task name hash. */
-    protected int taskNameHash;
-
-    /** */
-    @GridDirectTransient
-    private GridNearAtomicUpdateResponse res;
-
-    /** Compressed boolean flags. */
-    protected byte flags;
-
     /**
      * Empty constructor required by {@link Externalizable}.
      */
@@ -109,9 +50,7 @@ public abstract class GridNearAtomicAbstractSingleUpdateRequest extends GridNear
      *
      * @param cacheId Cache ID.
      * @param nodeId Node ID.
-     * @param futVer Future version.
-     * @param fastMap Fast map scheme flag.
-     * @param updateVer Update version set if fast map is performed.
+     * @param futId Future ID.
      * @param topVer Topology version.
      * @param topLocked Topology locked flag.
      * @param syncMode Synchronization mode.
@@ -121,15 +60,12 @@ public abstract class GridNearAtomicAbstractSingleUpdateRequest extends GridNear
      * @param taskNameHash Task name hash code.
      * @param skipStore Skip write-through to a persistent storage.
      * @param keepBinary Keep binary flag.
-     * @param clientReq Client node request flag.
      * @param addDepInfo Deployment info flag.
      */
     protected GridNearAtomicAbstractSingleUpdateRequest(
         int cacheId,
         UUID nodeId,
-        GridCacheVersion futVer,
-        boolean fastMap,
-        @Nullable GridCacheVersion updateVer,
+        long futId,
         @NotNull AffinityTopologyVersion topVer,
         boolean topLocked,
         CacheWriteSynchronizationMode syncMode,
@@ -137,91 +73,25 @@ public abstract class GridNearAtomicAbstractSingleUpdateRequest extends GridNear
         boolean retval,
         @Nullable UUID subjId,
         int taskNameHash,
+        boolean mappingKnown,
         boolean skipStore,
         boolean keepBinary,
-        boolean clientReq,
         boolean addDepInfo
     ) {
-        assert futVer != null;
-
-        this.cacheId = cacheId;
-        this.nodeId = nodeId;
-        this.futVer = futVer;
-        this.updateVer = updateVer;
-        this.topVer = topVer;
-        this.syncMode = syncMode;
-        this.op = op;
-        this.subjId = subjId;
-        this.taskNameHash = taskNameHash;
-        this.addDepInfo = addDepInfo;
-
-        fastMap(fastMap);
-        topologyLocked(topLocked);
-        returnValue(retval);
-        skipStore(skipStore);
-        keepBinary(keepBinary);
-        clientRequest(clientReq);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int lookupIndex() {
-        return CACHE_MSG_IDX;
-    }
-
-    /**
-     * @return Mapped node ID.
-     */
-    @Override public UUID nodeId() {
-        return nodeId;
-    }
-
-    /**
-     * @param nodeId Node ID.
-     */
-    @Override public void nodeId(UUID nodeId) {
-        this.nodeId = nodeId;
-    }
-
-    /**
-     * @return Subject ID.
-     */
-    @Override public UUID subjectId() {
-        return subjId;
-    }
-
-    /**
-     * @return Task name hash.
-     */
-    @Override public int taskNameHash() {
-        return taskNameHash;
-    }
-
-    /**
-     * @return Future version.
-     */
-    @Override public GridCacheVersion futureVersion() {
-        return futVer;
-    }
-
-    /**
-     * @return Update version for fast-map request.
-     */
-    @Override public GridCacheVersion updateVersion() {
-        return updateVer;
-    }
-
-    /**
-     * @return Topology version.
-     */
-    @Override public AffinityTopologyVersion topologyVersion() {
-        return topVer;
-    }
-
-    /**
-     * @return Cache write synchronization mode.
-     */
-    @Override public CacheWriteSynchronizationMode writeSynchronizationMode() {
-        return syncMode;
+        super(cacheId,
+            nodeId,
+            futId,
+            topVer,
+            topLocked,
+            syncMode,
+            op,
+            retval,
+            subjId,
+            taskNameHash,
+            mappingKnown,
+            skipStore,
+            keepBinary,
+            addDepInfo);
     }
 
     /**
@@ -232,331 +102,14 @@ public abstract class GridNearAtomicAbstractSingleUpdateRequest extends GridNear
     }
 
     /**
-     * @return Update operation.
-     */
-    @Override public GridCacheOperation operation() {
-        return op;
-    }
-
-    /**
      * @return Optional arguments for entry processor.
      */
     @Override @Nullable public Object[] invokeArguments() {
         return null;
     }
 
-    /**
-     * @param res Response.
-     * @return {@code True} if current response was {@code null}.
-     */
-    @Override public boolean onResponse(GridNearAtomicUpdateResponse res) {
-        if (this.res == null) {
-            this.res = res;
-
-            return true;
-        }
-
-        return false;
-    }
-
-    /**
-     * @return Response.
-     */
-    @Override @Nullable public GridNearAtomicUpdateResponse response() {
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean addDeploymentInfo() {
-        return addDepInfo;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteLogger messageLogger(GridCacheSharedContext ctx) {
-        return ctx.atomicMessageLogger();
-    }
-
-    /**
-     * @return Flag indicating whether this is fast-map udpate.
-     */
-    @Override public boolean fastMap() {
-        return isFlag(FAST_MAP_FLAG_MASK);
-    }
-
-    /**
-     * Sets fastMap flag value.
-     */
-    public void fastMap(boolean val) {
-        setFlag(val, FAST_MAP_FLAG_MASK);
-    }
-
-    /**
-     * @return Topology locked flag.
-     */
-    @Override public boolean topologyLocked() {
-        return isFlag(TOP_LOCKED_FLAG_MASK);
-    }
-
-    /**
-     * Sets topologyLocked flag value.
-     */
-    public void topologyLocked(boolean val) {
-        setFlag(val, TOP_LOCKED_FLAG_MASK);
-    }
-
-    /**
-     * @return {@code True} if request sent from client node.
-     */
-    @Override public boolean clientRequest() {
-        return isFlag(CLIENT_REQ_FLAG_MASK);
-    }
-
-    /**
-     * Sets clientRequest flag value.
-     */
-    public void clientRequest(boolean val) {
-        setFlag(val, CLIENT_REQ_FLAG_MASK);
-    }
-
-    /**
-     * @return Return value flag.
-     */
-    @Override public boolean returnValue() {
-        return isFlag(RET_VAL_FLAG_MASK);
-    }
-
-    /**
-     * Sets returnValue flag value.
-     */
-    public void returnValue(boolean val) {
-        setFlag(val, RET_VAL_FLAG_MASK);
-    }
-
-    /**
-     * @return Skip write-through to a persistent storage.
-     */
-    @Override public boolean skipStore() {
-        return isFlag(SKIP_STORE_FLAG_MASK);
-    }
-
-    /**
-     * Sets skipStore flag value.
-     */
-    public void skipStore(boolean val) {
-        setFlag(val, SKIP_STORE_FLAG_MASK);
-    }
-
-    /**
-     * @return Keep binary flag.
-     */
-    @Override public boolean keepBinary() {
-        return isFlag(KEEP_BINARY_FLAG_MASK);
-    }
-
-    /**
-     * Sets keepBinary flag value.
-     */
-    public void keepBinary(boolean val) {
-        setFlag(val, KEEP_BINARY_FLAG_MASK);
-    }
-
-    /**
-     * @return Flag indicating whether this request contains primary keys.
-     */
-    @Override public boolean hasPrimary() {
-        return isFlag(HAS_PRIMARY_FLAG_MASK);
-    }
-
-    /**
-     * Sets hasPrimary flag value.
-     */
-    public void hasPrimary(boolean val) {
-        setFlag(val, HAS_PRIMARY_FLAG_MASK);
-    }
-
     /** {@inheritDoc} */
     @Nullable @Override public CacheEntryPredicate[] filter() {
         return NO_FILTER;
     }
-
-    /**
-     * Sets flag mask.
-     *
-     * @param flag Set or clear.
-     * @param mask Mask.
-     */
-    private void setFlag(boolean flag, int mask) {
-        flags = flag ? (byte)(flags | mask) : (byte)(flags & ~mask);
-    }
-
-    /**
-     * Reads flag mask.
-     *
-     * @param mask Mask to read.
-     * @return Flag value.
-     */
-    private boolean isFlag(int mask) {
-        return (flags & mask) != 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
-        writer.setBuffer(buf);
-
-        if (!super.writeTo(buf, writer))
-            return false;
-
-        if (!writer.isHeaderWritten()) {
-            if (!writer.writeHeader(directType(), fieldsCount()))
-                return false;
-
-            writer.onHeaderWritten();
-        }
-
-        switch (writer.state()) {
-            case 3:
-                if (!writer.writeByte("flags", flags))
-                    return false;
-
-                writer.incrementState();
-
-            case 4:
-                if (!writer.writeMessage("futVer", futVer))
-                    return false;
-
-                writer.incrementState();
-
-            case 5:
-                if (!writer.writeByte("op", op != null ? (byte)op.ordinal() : -1))
-                    return false;
-
-                writer.incrementState();
-
-            case 6:
-                if (!writer.writeUuid("subjId", subjId))
-                    return false;
-
-                writer.incrementState();
-
-            case 7:
-                if (!writer.writeByte("syncMode", syncMode != null ? (byte)syncMode.ordinal() : -1))
-                    return false;
-
-                writer.incrementState();
-
-            case 8:
-                if (!writer.writeInt("taskNameHash", taskNameHash))
-                    return false;
-
-                writer.incrementState();
-
-            case 9:
-                if (!writer.writeMessage("topVer", topVer))
-                    return false;
-
-                writer.incrementState();
-
-            case 10:
-                if (!writer.writeMessage("updateVer", updateVer))
-                    return false;
-
-                writer.incrementState();
-
-        }
-
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
-        reader.setBuffer(buf);
-
-        if (!reader.beforeMessageRead())
-            return false;
-
-        if (!super.readFrom(buf, reader))
-            return false;
-
-        switch (reader.state()) {
-            case 3:
-                flags = reader.readByte("flags");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 4:
-                futVer = reader.readMessage("futVer");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 5:
-                byte opOrd;
-
-                opOrd = reader.readByte("op");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                op = GridCacheOperation.fromOrdinal(opOrd);
-
-                reader.incrementState();
-
-            case 6:
-                subjId = reader.readUuid("subjId");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 7:
-                byte syncModeOrd;
-
-                syncModeOrd = reader.readByte("syncMode");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                syncMode = CacheWriteSynchronizationMode.fromOrdinal(syncModeOrd);
-
-                reader.incrementState();
-
-            case 8:
-                taskNameHash = reader.readInt("taskNameHash");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 9:
-                topVer = reader.readMessage("topVer");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 10:
-                updateVer = reader.readMessage("updateVer");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-        }
-
-        return reader.afterMessageRead(GridNearAtomicAbstractSingleUpdateRequest.class);
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte fieldsCount() {
-        return 11;
-    }
 }


[7/8] ignite git commit: ignite-4705 Atomic cache protocol change: notify client node from backups

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateRequest.java
index deb9ce4..2826215 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateRequest.java
@@ -18,26 +18,65 @@
 package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 
 import java.io.Externalizable;
+import java.nio.ByteBuffer;
 import java.util.UUID;
 import javax.cache.processor.EntryProcessor;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheDeployable;
 import org.apache.ignite.internal.processors.cache.GridCacheMessage;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
 /**
  *
  */
 public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheMessage implements GridCacheDeployable {
+    /** Skip store flag bit mask. */
+    private static final int DHT_ATOMIC_SKIP_STORE_FLAG_MASK = 0x01;
+
+    /** Keep binary flag. */
+    private static final int DHT_ATOMIC_KEEP_BINARY_FLAG_MASK = 0x02;
+
+    /** Near cache key flag. */
+    private static final int DHT_ATOMIC_NEAR_FLAG_MASK = 0x04;
+
+    /** */
+    static final int DHT_ATOMIC_HAS_RESULT_MASK = 0x08;
+
+    /** */
+    private static final int DHT_ATOMIC_REPLY_WITHOUT_DELAY = 0x10;
+
     /** Message index. */
     public static final int CACHE_MSG_IDX = nextIndexId();
 
+    /** Future ID on primary. */
+    protected long futId;
+
+    /** Write version. */
+    protected GridCacheVersion writeVer;
+
+    /** Write synchronization mode. */
+    protected CacheWriteSynchronizationMode syncMode;
+
+    /** Topology version. */
+    protected AffinityTopologyVersion topVer;
+
+    /** Subject ID. */
+    protected UUID subjId;
+
+    /** Task name hash. */
+    protected int taskNameHash;
+
     /** Node ID. */
     @GridDirectTransient
     protected UUID nodeId;
@@ -46,6 +85,15 @@ public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheMessag
     @GridDirectTransient
     private boolean onRes;
 
+    /** */
+    private UUID nearNodeId;
+
+    /** */
+    private long nearFutId;
+
+    /** Additional flags. */
+    protected byte flags;
+
     /**
      * Empty constructor required by {@link Externalizable}.
      */
@@ -59,9 +107,68 @@ public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheMessag
      * @param cacheId Cache ID.
      * @param nodeId Node ID.
      */
-    protected GridDhtAtomicAbstractUpdateRequest(int cacheId, UUID nodeId) {
+    protected GridDhtAtomicAbstractUpdateRequest(int cacheId,
+        UUID nodeId,
+        long futId,
+        GridCacheVersion writeVer,
+        CacheWriteSynchronizationMode syncMode,
+        @NotNull AffinityTopologyVersion topVer,
+        UUID subjId,
+        int taskNameHash,
+        boolean addDepInfo,
+        boolean keepBinary,
+        boolean skipStore
+    ) {
         this.cacheId = cacheId;
         this.nodeId = nodeId;
+        this.futId = futId;
+        this.writeVer = writeVer;
+        this.syncMode = syncMode;
+        this.topVer = topVer;
+        this.subjId = subjId;
+        this.taskNameHash = taskNameHash;
+        this.addDepInfo = addDepInfo;
+
+        if (skipStore)
+            setFlag(true, DHT_ATOMIC_SKIP_STORE_FLAG_MASK);
+        if (keepBinary)
+            setFlag(true, DHT_ATOMIC_KEEP_BINARY_FLAG_MASK);
+    }
+
+    void nearReplyInfo(UUID nearNodeId, long nearFutId) {
+        assert nearNodeId != null;
+
+        this.nearNodeId = nearNodeId;
+        this.nearFutId = nearFutId;
+    }
+
+    boolean replyWithoutDelay() {
+        return isFlag(DHT_ATOMIC_REPLY_WITHOUT_DELAY);
+    }
+
+    void replyWithoutDelay(boolean replyWithoutDelay) {
+        setFlag(replyWithoutDelay, DHT_ATOMIC_REPLY_WITHOUT_DELAY);
+    }
+
+    /**
+     * @param res Result flag.
+     */
+    void hasResult(boolean res) {
+        setFlag(res, DHT_ATOMIC_HAS_RESULT_MASK);
+    }
+
+    /**
+     * @return Result flag.
+     */
+    private boolean hasResult() {
+        return isFlag(DHT_ATOMIC_HAS_RESULT_MASK);
+    }
+
+    /**
+     * @return Near node ID.
+     */
+    public UUID nearNodeId() {
+        return nearNodeId;
     }
 
     /** {@inheritDoc} */
@@ -77,14 +184,25 @@ public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheMessag
     }
 
     /**
+     * @return Flags.
+     */
+    public final byte flags() {
+        return flags;
+    }
+
+    /**
      * @return Keep binary flag.
      */
-    public abstract boolean keepBinary();
+    public final boolean keepBinary() {
+        return isFlag(DHT_ATOMIC_KEEP_BINARY_FLAG_MASK);
+    }
 
     /**
      * @return Skip write-through to a persistent storage.
      */
-    public abstract boolean skipStore();
+    public final boolean skipStore() {
+        return isFlag(DHT_ATOMIC_SKIP_STORE_FLAG_MASK);
+    }
 
     /**
      * @return {@code True} if on response flag changed.
@@ -93,6 +211,13 @@ public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheMessag
         return !onRes && (onRes = true);
     }
 
+    /**
+     * @return {@code True} if response was received.
+     */
+    boolean hasResponse() {
+        return onRes;
+    }
+
     /** {@inheritDoc} */
     @Override public boolean addDeploymentInfo() {
         return addDepInfo;
@@ -121,7 +246,6 @@ public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheMessag
      * @param conflictExpireTime Conflict expire time (optional).
      * @param conflictVer Conflict version (optional).
      * @param addPrevVal If {@code true} adds previous value.
-     * @param partId Partition.
      * @param prevVal Previous value.
      * @param updateCntr Update counter.
      */
@@ -132,7 +256,6 @@ public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheMessag
         long conflictExpireTime,
         @Nullable GridCacheVersion conflictVer,
         boolean addPrevVal,
-        int partId,
         @Nullable CacheObject prevVal,
         long updateCntr
     );
@@ -158,27 +281,44 @@ public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheMessag
     /**
      * @return Subject ID.
      */
-    public abstract UUID subjectId();
+    public final UUID subjectId() {
+        return subjId;
+    }
 
     /**
      * @return Task name.
      */
-    public abstract int taskNameHash();
+    public final int taskNameHash() {
+        return taskNameHash;
+    }
+
+    /**
+     * @return Future ID on primary node.
+     */
+    public final long futureId() {
+        return futId;
+    }
 
     /**
-     * @return Version assigned on primary node.
+     * @return Future ID on near node.
      */
-    public abstract GridCacheVersion futureVersion();
+    public final long nearFutureId() {
+        return nearFutId;
+    }
 
     /**
      * @return Write version.
      */
-    public abstract GridCacheVersion writeVersion();
+    public final GridCacheVersion writeVersion() {
+        return writeVer;
+    }
 
     /**
      * @return Cache write synchronization mode.
      */
-    public abstract CacheWriteSynchronizationMode writeSynchronizationMode();
+    public final CacheWriteSynchronizationMode writeSynchronizationMode() {
+        return syncMode;
+    }
 
     /**
      * @return Keys size.
@@ -203,12 +343,6 @@ public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheMessag
     public abstract KeyCacheObject key(int idx);
 
     /**
-     * @param idx Partition index.
-     * @return Partition id.
-     */
-    public abstract int partitionId(int idx);
-
-    /**
      * @param updCntr Update counter.
      * @return Update counter.
      */
@@ -284,4 +418,228 @@ public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheMessag
      * @return Optional arguments for entry processor.
      */
     @Nullable public abstract Object[] invokeArguments();
+
+    /**
+     * @return {@code True} if near cache update request.
+     */
+    protected final boolean near() {
+        return isFlag(DHT_ATOMIC_NEAR_FLAG_MASK);
+    }
+
+    /**
+     * @param near Near cache update flag.
+     */
+    protected final void near(boolean near) {
+        setFlag(near, DHT_ATOMIC_NEAR_FLAG_MASK);
+    }
+
+    /**
+     * Sets flag mask.
+     *
+     * @param flag Set or clear.
+     * @param mask Mask.
+     */
+    protected final void setFlag(boolean flag, int mask) {
+        flags = flag ? (byte)(flags | mask) : (byte)(flags & ~mask);
+    }
+
+    /**
+     * Reags flag mask.
+     *
+     * @param mask Mask to read.
+     * @return Flag value.
+     */
+    final boolean isFlag(int mask) {
+        return (flags & mask) != 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 12;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 3:
+                if (!writer.writeByte("flags", flags))
+                    return false;
+
+                writer.incrementState();
+
+            case 4:
+                if (!writer.writeLong("futId", futId))
+                    return false;
+
+                writer.incrementState();
+
+            case 5:
+                if (!writer.writeLong("nearFutId", nearFutId))
+                    return false;
+
+                writer.incrementState();
+
+            case 6:
+                if (!writer.writeUuid("nearNodeId", nearNodeId))
+                    return false;
+
+                writer.incrementState();
+
+            case 7:
+                if (!writer.writeUuid("subjId", subjId))
+                    return false;
+
+                writer.incrementState();
+
+            case 8:
+                if (!writer.writeByte("syncMode", syncMode != null ? (byte)syncMode.ordinal() : -1))
+                    return false;
+
+                writer.incrementState();
+
+            case 9:
+                if (!writer.writeInt("taskNameHash", taskNameHash))
+                    return false;
+
+                writer.incrementState();
+
+            case 10:
+                if (!writer.writeMessage("topVer", topVer))
+                    return false;
+
+                writer.incrementState();
+
+            case 11:
+                if (!writer.writeMessage("writeVer", writeVer))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 3:
+                flags = reader.readByte("flags");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 4:
+                futId = reader.readLong("futId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 5:
+                nearFutId = reader.readLong("nearFutId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 6:
+                nearNodeId = reader.readUuid("nearNodeId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 7:
+                subjId = reader.readUuid("subjId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 8:
+                byte syncModeOrd;
+
+                syncModeOrd = reader.readByte("syncMode");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                syncMode = CacheWriteSynchronizationMode.fromOrdinal(syncModeOrd);
+
+                reader.incrementState();
+
+            case 9:
+                taskNameHash = reader.readInt("taskNameHash");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 10:
+                topVer = reader.readMessage("topVer");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 11:
+                writeVer = reader.readMessage("writeVer");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(GridDhtAtomicAbstractUpdateRequest.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        StringBuilder flags = new StringBuilder();
+
+        if (skipStore())
+            appendFlag(flags, "skipStore");
+        if (keepBinary())
+            appendFlag(flags, "keepBinary");
+        if (near())
+            appendFlag(flags, "near");
+        if (hasResult())
+            appendFlag(flags, "hasRes");
+        if (replyWithoutDelay())
+            appendFlag(flags, "resNoDelay");
+
+        return S.toString(GridDhtAtomicAbstractUpdateRequest.class, this,
+            "flags", flags.toString());
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index cebf4ae..c20ed48 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -38,6 +38,7 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.affinity.AffinityAssignment;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheEntryPredicate;
 import org.apache.ignite.internal.processors.cache.CacheInvokeEntry;
@@ -57,7 +58,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheMapEntryFactory;
 import org.apache.ignite.internal.processors.cache.GridCacheOperation;
 import org.apache.ignite.internal.processors.cache.GridCacheReturn;
 import org.apache.ignite.internal.processors.cache.GridCacheUpdateAtomicResult;
-import org.apache.ignite.internal.processors.cache.GridDeferredAckMessageSender;
 import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
@@ -80,6 +80,8 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionEx;
+import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
+import org.apache.ignite.internal.util.GridLongList;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.future.GridEmbeddedFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
@@ -90,8 +92,6 @@ import org.apache.ignite.internal.util.typedef.CI2;
 import org.apache.ignite.internal.util.typedef.CO;
 import org.apache.ignite.internal.util.typedef.CX1;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.P1;
-import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -99,16 +99,14 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteOutClosure;
+import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.plugin.security.SecurityPermission;
 import org.apache.ignite.transactions.TransactionIsolation;
 import org.jetbrains.annotations.Nullable;
-import org.jsr166.ConcurrentLinkedDeque8;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_ATOMIC_DEFERRED_ACK_BUFFER_SIZE;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_ATOMIC_DEFERRED_ACK_TIMEOUT;
-import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.DELETE;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.UPDATE;
@@ -134,12 +132,17 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     private static final int DEFERRED_UPDATE_RESPONSE_TIMEOUT =
         Integer.getInteger(IGNITE_ATOMIC_DEFERRED_ACK_TIMEOUT, 500);
 
+    /** */
+    private final ThreadLocal<Map<UUID, GridDhtAtomicDeferredUpdateResponse>> defRes =
+        new ThreadLocal<Map<UUID, GridDhtAtomicDeferredUpdateResponse>>() {
+            @Override protected Map<UUID, GridDhtAtomicDeferredUpdateResponse> initialValue() {
+                return new HashMap<>();
+            }
+        };
+
     /** Update reply closure. */
     @GridToStringExclude
-    private CI2<GridNearAtomicAbstractUpdateRequest, GridNearAtomicUpdateResponse> updateReplyClos;
-
-    /** Pending */
-    private GridDeferredAckMessageSender deferredUpdateMsgSnd;
+    private UpdateReplyClosure updateReplyClos;
 
     /** */
     private GridNearAtomicCache<K, V> near;
@@ -205,25 +208,13 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @Override protected void init() {
         super.init();
 
-        updateReplyClos = new CI2<GridNearAtomicAbstractUpdateRequest, GridNearAtomicUpdateResponse>() {
+        updateReplyClos = new UpdateReplyClosure() {
             @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
             @Override public void apply(GridNearAtomicAbstractUpdateRequest req, GridNearAtomicUpdateResponse res) {
-                if (ctx.config().getAtomicWriteOrderMode() == CLOCK) {
-                    assert req.writeSynchronizationMode() != FULL_ASYNC : req;
-
-                    // Always send reply in CLOCK ordering mode.
-                    sendNearUpdateReply(res.nodeId(), res);
-
-                    return;
-                }
-
-                // Request should be for primary keys only in PRIMARY ordering mode.
-                assert req.hasPrimary() : req;
-
                 if (req.writeSynchronizationMode() != FULL_ASYNC)
                     sendNearUpdateReply(res.nodeId(), res);
                 else {
-                    if (!F.isEmpty(res.remapKeys()))
+                    if (res.remapTopologyVersion() != null)
                         // Remap keys on primary node in FULL_ASYNC mode.
                         remapToNewPrimary(req);
                     else if (res.error() != null) {
@@ -240,53 +231,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @Override public void start() throws IgniteCheckedException {
         super.start();
 
-        deferredUpdateMsgSnd = new GridDeferredAckMessageSender(ctx.time(), ctx.closures()) {
-            @Override public int getTimeout() {
-                return DEFERRED_UPDATE_RESPONSE_TIMEOUT;
-            }
-
-            @Override public int getBufferSize() {
-                return DEFERRED_UPDATE_RESPONSE_BUFFER_SIZE;
-            }
-
-            @Override public void finish(UUID nodeId, ConcurrentLinkedDeque8<GridCacheVersion> vers) {
-                GridDhtAtomicDeferredUpdateResponse msg = new GridDhtAtomicDeferredUpdateResponse(ctx.cacheId(),
-                    vers, ctx.deploymentEnabled());
-
-                try {
-                    ctx.kernalContext().gateway().readLock();
-
-                    try {
-                        ctx.io().send(nodeId, msg, ctx.ioPolicy());
-
-                        if (msgLog.isDebugEnabled()) {
-                            msgLog.debug("Sent deferred DHT update response [futIds=" + msg.futureVersions() +
-                                ", node=" + nodeId + ']');
-                        }
-                    }
-                    finally {
-                        ctx.kernalContext().gateway().readUnlock();
-                    }
-                }
-                catch (IllegalStateException ignored) {
-                    if (msgLog.isDebugEnabled()) {
-                        msgLog.debug("Failed to send deferred DHT update response, node is stopping [" +
-                            "futIds=" + msg.futureVersions() + ", node=" + nodeId + ']');
-                    }
-                }
-                catch (ClusterTopologyCheckedException ignored) {
-                    if (msgLog.isDebugEnabled()) {
-                        msgLog.debug("Failed to send deferred DHT update response, node left [" +
-                            "futIds=" + msg.futureVersions() + ", node=" + nodeId + ']');
-                    }
-                }
-                catch (IgniteCheckedException e) {
-                    U.error(log, "Failed to send deferred DHT update response to remote node [" +
-                        "futIds=" + msg.futureVersions() + ", node=" + nodeId + ']', e);
-                }
-            }
-        };
-
         CacheMetricsImpl m = new CacheMetricsImpl(ctx);
 
         if (ctx.dht().near() != null)
@@ -419,6 +363,32 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                 }
             });
 
+        ctx.io().addHandler(ctx.cacheId(),
+            GridDhtAtomicNearResponse.class,
+            new CI2<UUID, GridDhtAtomicNearResponse>() {
+            @Override public void apply(UUID uuid, GridDhtAtomicNearResponse msg) {
+                processDhtAtomicNearResponse(uuid, msg);
+            }
+
+            @Override public String toString() {
+                return "GridDhtAtomicNearResponse handler " +
+                    "[msgIdx=" + GridDhtAtomicNearResponse.CACHE_MSG_IDX + ']';
+            }
+        });
+
+        ctx.io().addHandler(ctx.cacheId(),
+            GridNearAtomicCheckUpdateRequest.class,
+            new CI2<UUID, GridNearAtomicCheckUpdateRequest>() {
+                @Override public void apply(UUID uuid, GridNearAtomicCheckUpdateRequest msg) {
+                    processCheckUpdateRequest(uuid, msg);
+                }
+
+                @Override public String toString() {
+                    return "GridNearAtomicCheckUpdateRequest handler " +
+                        "[msgIdx=" + GridNearAtomicCheckUpdateRequest.CACHE_MSG_IDX + ']';
+                }
+            });
+
         if (near == null) {
             ctx.io().addHandler(
                 ctx.cacheId(),
@@ -450,11 +420,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         }
     }
 
-    /** {@inheritDoc} */
-    @Override public void stop() {
-        deferredUpdateMsgSnd.stop();
-    }
-
     /**
      * @param near Near cache.
      */
@@ -1341,9 +1306,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
         CacheEntryPredicate[] filters = CU.filterArray(filter);
 
-        if (conflictPutVal == null &&
-            conflictRmvVer == null &&
-            !isFastMap(filters, op)) {
+        if (conflictPutVal == null && conflictRmvVer == null) {
             return new GridNearAtomicSingleUpdateFuture(
                 ctx,
                 this,
@@ -1389,19 +1352,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /**
-     * Whether this is fast-map operation.
-     *
-     * @param filters Filters.
-     * @param op Operation.
-     * @return {@code True} if fast-map.
-     */
-    public boolean isFastMap(CacheEntryPredicate[] filters, GridCacheOperation op) {
-        return F.isEmpty(filters) && op != TRANSFORM && ctx.config().getWriteSynchronizationMode() == FULL_SYNC &&
-            ctx.config().getAtomicWriteOrderMode() == CLOCK &&
-            !(ctx.writeThrough() && ctx.config().getInterceptor() != null);
-    }
-
-    /**
      * Entry point for all public API remove methods.
      *
      * @param keys Keys to remove.
@@ -1696,10 +1646,10 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @param req Update request.
      * @param completionCb Completion callback.
      */
-    public void updateAllAsyncInternal(
+    void updateAllAsyncInternal(
         final UUID nodeId,
         final GridNearAtomicAbstractUpdateRequest req,
-        final CI2<GridNearAtomicAbstractUpdateRequest, GridNearAtomicUpdateResponse> completionCb
+        final UpdateReplyClosure completionCb
     ) {
         IgniteInternalFuture<Object> forceFut = preldr.request(req, req.topologyVersion());
 
@@ -1748,12 +1698,14 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      */
     private void onForceKeysError(final UUID nodeId,
         final GridNearAtomicAbstractUpdateRequest req,
-        final CI2<GridNearAtomicAbstractUpdateRequest, GridNearAtomicUpdateResponse> completionCb,
+        final UpdateReplyClosure completionCb,
         IgniteCheckedException e
     ) {
         GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(ctx.cacheId(),
             nodeId,
-            req.futureVersion(),
+            req.futureId(),
+            req.partition(),
+            false,
             ctx.deploymentEnabled());
 
         res.addFailedKeys(req.keys(), e);
@@ -1771,12 +1723,23 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     private void updateAllAsyncInternal0(
         UUID nodeId,
         GridNearAtomicAbstractUpdateRequest req,
-        CI2<GridNearAtomicAbstractUpdateRequest, GridNearAtomicUpdateResponse> completionCb
+        UpdateReplyClosure completionCb
     ) {
-        GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(ctx.cacheId(), nodeId, req.futureVersion(),
-            ctx.deploymentEnabled());
+        ClusterNode node = ctx.discovery().node(nodeId);
+
+        if (node == null) {
+            U.warn(msgLog, "Skip near update request, node originated update request left [" +
+                "futId=" + req.futureId() + ", node=" + nodeId + ']');
 
-        res.partition(req.partition());
+            return;
+        }
+
+        GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(ctx.cacheId(),
+            nodeId,
+            req.futureId(),
+            req.partition(),
+            false,
+            ctx.deploymentEnabled());
 
         assert !req.returnValue() || (req.operation() == TRANSFORM || req.size() == 1);
 
@@ -1791,7 +1754,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         try {
             // If batch store update is enabled, we need to lock all entries.
             // First, need to acquire locks on cache entries, then check filter.
-            List<GridDhtCacheEntry> locked = lockEntries(req, req.topologyVersion());
+            List<GridDhtCacheEntry> locked = null;
 
             Collection<IgniteBiTuple<GridDhtCacheEntry, GridCacheVersion>> deleted = null;
 
@@ -1810,43 +1773,29 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                         return;
                     }
 
-                    // Do not check topology version for CLOCK versioning since
-                    // partition exchange will wait for near update future (if future is on server node).
-                    // Also do not check topology version if topology was locked on near node by
+                    // Do not check topology version if topology was locked on near node by
                     // external transaction or explicit lock.
-                    if ((req.fastMap() && !req.clientRequest()) || req.topologyLocked() ||
-                        !needRemap(req.topologyVersion(), top.topologyVersion())) {
-                        ClusterNode node = ctx.discovery().node(nodeId);
-
-                        if (node == null) {
-                            U.warn(msgLog, "Skip near update request, node originated update request left [" +
-                                "futId=" + req.futureVersion() + ", node=" + nodeId + ']');
-
-                            return;
-                        }
+                    if (req.topologyLocked() || !needRemap(req.topologyVersion(), top.topologyVersion())) {
+                        locked = lockEntries(req, req.topologyVersion());
 
                         boolean hasNear = ctx.discovery().cacheNearNode(node, name());
 
-                        GridCacheVersion ver = req.updateVersion();
+                        // Assign next version for update inside entries lock.
+                        GridCacheVersion ver = ctx.versions().next(top.topologyVersion());
 
-                        if (ver == null) {
-                            // Assign next version for update inside entries lock.
-                            ver = ctx.versions().next(top.topologyVersion());
+                        if (hasNear)
+                            res.nearVersion(ver);
 
-                            if (hasNear)
-                                res.nearVersion(ver);
-
-                            if (msgLog.isDebugEnabled()) {
-                                msgLog.debug("Assigned update version [futId=" + req.futureVersion() +
-                                    ", writeVer=" + ver + ']');
-                            }
+                        if (msgLog.isDebugEnabled()) {
+                            msgLog.debug("Assigned update version [futId=" + req.futureId() +
+                                ", writeVer=" + ver + ']');
                         }
 
                         assert ver != null : "Got null version for update request: " + req;
 
                         boolean sndPrevVal = !top.rebalanceFinished(req.topologyVersion());
 
-                        dhtFut = createDhtFuture(ver, req, res, completionCb, false);
+                        dhtFut = createDhtFuture(ver, req);
 
                         expiry = expiryPolicy(req.expiry());
 
@@ -1866,7 +1815,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                                 locked,
                                 ver,
                                 dhtFut,
-                                completionCb,
                                 ctx.isDrEnabled(),
                                 taskName,
                                 expiry,
@@ -1886,7 +1834,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                                 locked,
                                 ver,
                                 dhtFut,
-                                completionCb,
                                 ctx.isDrEnabled(),
                                 taskName,
                                 expiry,
@@ -1902,15 +1849,15 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
                         res.returnValue(retVal);
 
-                        if (req.writeSynchronizationMode() != FULL_ASYNC)
-                            req.cleanup(!node.isLocal());
-
                         if (dhtFut != null)
-                            ctx.mvcc().addAtomicFuture(dhtFut.version(), dhtFut);
+                            ctx.mvcc().addAtomicFuture(dhtFut.id(), dhtFut);
                     }
-                    else
+                    else {
                         // Should remap all keys.
                         remap = true;
+
+                        res.remapTopologyVersion(top.topologyVersion());
+                    }
                 }
                 finally {
                     top.readUnlock();
@@ -1936,12 +1883,12 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             }
         }
         catch (GridDhtInvalidPartitionException ignore) {
-            assert !req.fastMap() || req.clientRequest() : req;
-
             if (log.isDebugEnabled())
                 log.debug("Caught invalid partition exception for cache entry (will remap update request): " + req);
 
             remap = true;
+
+            res.remapTopologyVersion(ctx.topology().topologyVersion());
         }
         catch (Throwable e) {
             // At least RuntimeException can be thrown by the code above when GridCacheContext is cleaned and there is
@@ -1961,18 +1908,14 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         if (remap) {
             assert dhtFut == null;
 
-            res.remapKeys(req.keys());
-
             completionCb.apply(req, res);
         }
-        else {
-            // If there are backups, map backup update future.
+        else
             if (dhtFut != null)
-                dhtFut.map();
-                // Otherwise, complete the call.
-            else
-                completionCb.apply(req, res);
-        }
+                dhtFut.map(node, res.returnValue(), res, completionCb);
+
+        if (req.writeSynchronizationMode() != FULL_ASYNC)
+            req.cleanup(!node.isLocal());
 
         sendTtlUpdateRequest(expiry);
     }
@@ -1987,7 +1930,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @param locked Locked entries.
      * @param ver Assigned version.
      * @param dhtFut Optional DHT future.
-     * @param completionCb Completion callback to invoke when DHT future is completed.
      * @param replicate Whether replication is enabled.
      * @param taskName Task name.
      * @param expiry Expiry policy.
@@ -2004,7 +1946,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         final List<GridDhtCacheEntry> locked,
         final GridCacheVersion ver,
         @Nullable GridDhtAtomicAbstractUpdateFuture dhtFut,
-        final CI2<GridNearAtomicAbstractUpdateRequest, GridNearAtomicUpdateResponse> completionCb,
         final boolean replicate,
         final String taskName,
         @Nullable final IgniteCacheExpiryPolicy expiry,
@@ -2049,9 +1990,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         for (int i = 0; i < locked.size(); i++) {
             GridDhtCacheEntry entry = locked.get(i);
 
-            if (entry == null)
-                continue;
-
             try {
                 if (!checkFilter(entry, req, res)) {
                     if (expiry != null && entry.hasValue()) {
@@ -2155,7 +2093,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                                 null,
                                 entryProcessorMap,
                                 dhtFut,
-                                completionCb,
                                 req,
                                 res,
                                 replicate,
@@ -2204,7 +2141,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                                 rmvKeys,
                                 entryProcessorMap,
                                 dhtFut,
-                                completionCb,
                                 req,
                                 res,
                                 replicate,
@@ -2331,7 +2267,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                 rmvKeys,
                 entryProcessorMap,
                 dhtFut,
-                completionCb,
                 req,
                 res,
                 replicate,
@@ -2404,14 +2339,13 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     /**
      * Updates locked entries one-by-one.
      *
-     * @param node Originating node.
+     * @param nearNode Originating node.
      * @param hasNear {@code True} if originating node has near cache.
      * @param req Update request.
      * @param res Update response.
      * @param locked Locked entries.
      * @param ver Assigned update version.
      * @param dhtFut Optional DHT future.
-     * @param completionCb Completion callback to invoke when DHT future is completed.
      * @param replicate Whether DR is enabled for that cache.
      * @param taskName Task name.
      * @param expiry Expiry policy.
@@ -2420,14 +2354,13 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @throws GridCacheEntryRemovedException Should be never thrown.
      */
     private UpdateSingleResult updateSingle(
-        ClusterNode node,
+        ClusterNode nearNode,
         boolean hasNear,
         GridNearAtomicAbstractUpdateRequest req,
         GridNearAtomicUpdateResponse res,
         List<GridDhtCacheEntry> locked,
         GridCacheVersion ver,
         @Nullable GridDhtAtomicAbstractUpdateFuture dhtFut,
-        CI2<GridNearAtomicAbstractUpdateRequest, GridNearAtomicUpdateResponse> completionCb,
         boolean replicate,
         String taskName,
         @Nullable IgniteCacheExpiryPolicy expiry,
@@ -2440,10 +2373,10 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
         boolean checkReaders = hasNear || ctx.discovery().hasNearCache(ctx.cacheId(), topVer);
 
-        boolean readersOnly = false;
-
         boolean intercept = ctx.config().getInterceptor() != null;
 
+        AffinityAssignment affAssignment = ctx.affinity().assignment(topVer);
+
         // Avoid iterator creation.
         for (int i = 0; i < req.size(); i++) {
             KeyCacheObject k = req.key(i);
@@ -2455,18 +2388,12 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             try {
                 GridDhtCacheEntry entry = locked.get(i);
 
-                if (entry == null)
-                    continue;
-
                 GridCacheVersion newConflictVer = req.conflictVersion(i);
                 long newConflictTtl = req.conflictTtl(i);
                 long newConflictExpireTime = req.conflictExpireTime(i);
 
                 assert !(newConflictVer instanceof GridCacheVersionEx) : newConflictVer;
 
-                boolean primary = !req.fastMap() || ctx.affinity().primaryByPartition(ctx.localNode(), entry.partition(),
-                    req.topologyVersion());
-
                 Object writeVal = op == TRANSFORM ? req.entryProcessor(i) : req.writeValue(i);
 
                 Collection<UUID> readers = null;
@@ -2474,46 +2401,39 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
                 if (checkReaders) {
                     readers = entry.readers();
-                    filteredReaders = F.view(entry.readers(), F.notEqualTo(node.id()));
+                    filteredReaders = F.view(entry.readers(), F.notEqualTo(nearNode.id()));
                 }
 
                 GridCacheUpdateAtomicResult updRes = entry.innerUpdate(
                     ver,
-                    node.id(),
+                    nearNode.id(),
                     locNodeId,
                     op,
                     writeVal,
                     req.invokeArguments(),
-                    (primary || (ctx.store().isLocal() && !ctx.shared().localStorePrimaryOnly()))
-                        && writeThrough() && !req.skipStore(),
+                    writeThrough() && !req.skipStore(),
                     !req.skipStore(),
                     sndPrevVal || req.returnValue(),
                     req.keepBinary(),
                     expiry,
-                    true,
-                    true,
-                    primary,
-                    ctx.config().getAtomicWriteOrderMode() == CLOCK, // Check version in CLOCK mode on primary node.
+                    /*event*/true,
+                    /*metrics*/true,
+                    /*primary*/true,
+                    /*verCheck*/false,
                     topVer,
                     req.filter(),
-                    replicate ? primary ? DR_PRIMARY : DR_BACKUP : DR_NONE,
+                    replicate ? DR_PRIMARY : DR_NONE,
                     newConflictTtl,
                     newConflictExpireTime,
                     newConflictVer,
-                    true,
+                    /*conflictResolve*/true,
                     intercept,
                     req.subjectId(),
                     taskName,
-                    null,
-                    null,
+                    /*prevVal*/null,
+                    /*updateCntr*/null,
                     dhtFut);
 
-                if (dhtFut == null && !F.isEmpty(filteredReaders)) {
-                    dhtFut = createDhtFuture(ver, req, res, completionCb, true);
-
-                    readersOnly = true;
-                }
-
                 if (dhtFut != null) {
                     if (updRes.sendToDht()) { // Send to backups even in case of remove-remove scenarios.
                         GridCacheVersionConflictContext<?, ?> conflictCtx = updRes.conflictResolveResult();
@@ -2525,20 +2445,21 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
                         EntryProcessor<Object, Object, Object> entryProcessor = null;
 
-                        if (!readersOnly) {
-                            dhtFut.addWriteEntry(entry,
-                                updRes.newValue(),
-                                entryProcessor,
-                                updRes.newTtl(),
-                                updRes.conflictExpireTime(),
-                                newConflictVer,
-                                sndPrevVal,
-                                updRes.oldValue(),
-                                updRes.updateCounter());
-                        }
+                        dhtFut.addWriteEntry(
+                            affAssignment,
+                            entry,
+                            updRes.newValue(),
+                            entryProcessor,
+                            updRes.newTtl(),
+                            updRes.conflictExpireTime(),
+                            newConflictVer,
+                            sndPrevVal,
+                            updRes.oldValue(),
+                            updRes.updateCounter());
 
                         if (!F.isEmpty(filteredReaders))
-                            dhtFut.addNearWriteEntries(filteredReaders,
+                            dhtFut.addNearWriteEntries(
+                                filteredReaders,
                                 entry,
                                 updRes.newValue(),
                                 entryProcessor,
@@ -2553,8 +2474,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                 }
 
                 if (hasNear) {
-                    if (primary && updRes.sendToDht()) {
-                        if (!ctx.affinity().partitionBelongs(node, entry.partition(), topVer)) {
+                    if (updRes.sendToDht()) {
+                        if (!ctx.affinity().partitionBelongs(nearNode, entry.partition(), topVer)) {
                             // If put the same value as in request then do not need to send it back.
                             if (op == TRANSFORM || writeVal != updRes.newValue()) {
                                 res.addNearValue(i,
@@ -2566,13 +2487,13 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                                 res.addNearTtl(i, updRes.newTtl(), updRes.conflictExpireTime());
 
                             if (updRes.newValue() != null) {
-                                IgniteInternalFuture<Boolean> f = entry.addReader(node.id(), req.messageId(), topVer);
+                                IgniteInternalFuture<Boolean> f = entry.addReader(nearNode.id(), req.messageId(), topVer);
 
                                 assert f == null : f;
                             }
                         }
-                        else if (F.contains(readers, node.id())) // Reader became primary or backup.
-                            entry.removeReader(node.id(), req.messageId());
+                        else if (F.contains(readers, nearNode.id())) // Reader became primary or backup.
+                            entry.removeReader(nearNode.id(), req.messageId());
                         else
                             res.addSkippedIndex(i);
                     }
@@ -2594,7 +2515,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
                     if (compRes != null && (compRes.get1() != null || compRes.get2() != null)) {
                         if (retVal == null)
-                            retVal = new GridCacheReturn(node.isLocal());
+                            retVal = new GridCacheReturn(nearNode.isLocal());
 
                         retVal.addEntryProcessResult(ctx,
                             k,
@@ -2610,7 +2531,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                         CacheObject ret = updRes.oldValue();
 
                         retVal = new GridCacheReturn(ctx,
-                            node.isLocal(),
+                            nearNode.isLocal(),
                             req.keepBinary(),
                             req.returnValue() ? ret : null,
                             updRes.success());
@@ -2630,13 +2551,12 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @param firstEntryIdx Index of the first entry in the request keys collection.
      * @param entries Entries to update.
      * @param ver Version to set.
-     * @param node Originating node.
+     * @param nearNode Originating node.
      * @param writeVals Write values.
      * @param putMap Values to put.
      * @param rmvKeys Keys to remove.
      * @param entryProcessorMap Entry processors.
      * @param dhtFut DHT update future if has backups.
-     * @param completionCb Completion callback to invoke when DHT future is completed.
      * @param req Request.
      * @param res Response.
      * @param replicate Whether replication is enabled.
@@ -2652,13 +2572,12 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         final int firstEntryIdx,
         final List<GridDhtCacheEntry> entries,
         final GridCacheVersion ver,
-        final ClusterNode node,
+        final ClusterNode nearNode,
         @Nullable final List<CacheObject> writeVals,
         @Nullable final Map<KeyCacheObject, CacheObject> putMap,
         @Nullable final Collection<KeyCacheObject> rmvKeys,
         @Nullable final Map<KeyCacheObject, EntryProcessor<Object, Object, Object>> entryProcessorMap,
         @Nullable GridDhtAtomicAbstractUpdateFuture dhtFut,
-        final CI2<GridNearAtomicAbstractUpdateRequest, GridNearAtomicUpdateResponse> completionCb,
         final GridNearAtomicAbstractUpdateRequest req,
         final GridNearAtomicUpdateResponse res,
         final boolean replicate,
@@ -2681,17 +2600,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             GridCacheOperation op;
 
             if (putMap != null) {
-                // If fast mapping, filter primary keys for write to store.
-                Map<KeyCacheObject, CacheObject> storeMap = req.fastMap() ?
-                    F.view(putMap, new P1<CacheObject>() {
-                        @Override public boolean apply(CacheObject key) {
-                            return ctx.affinity().primaryByKey(ctx.localNode(), key, req.topologyVersion());
-                        }
-                    }) :
-                    putMap;
-
                 try {
-                    ctx.store().putAll(null, F.viewReadOnly(storeMap, new C1<CacheObject, IgniteBiTuple<CacheObject, GridCacheVersion>>() {
+                    ctx.store().putAll(null, F.viewReadOnly(putMap, new C1<CacheObject, IgniteBiTuple<CacheObject, GridCacheVersion>>() {
                         @Override public IgniteBiTuple<CacheObject, GridCacheVersion> apply(CacheObject v) {
                             return F.t(v, ver);
                         }
@@ -2704,17 +2614,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                 op = UPDATE;
             }
             else {
-                // If fast mapping, filter primary keys for write to store.
-                Collection<KeyCacheObject> storeKeys = req.fastMap() ?
-                    F.view(rmvKeys, new P1<Object>() {
-                        @Override public boolean apply(Object key) {
-                            return ctx.affinity().primaryByKey(ctx.localNode(), key, req.topologyVersion());
-                        }
-                    }) :
-                    rmvKeys;
-
                 try {
-                    ctx.store().removeAll(null, storeKeys);
+                    ctx.store().removeAll(null, rmvKeys);
                 }
                 catch (CacheStorePartialUpdateException e) {
                     storeErr = e;
@@ -2725,6 +2626,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
             boolean intercept = ctx.config().getInterceptor() != null;
 
+            AffinityAssignment affAssignment = ctx.affinity().assignment(topVer);
+
             // Avoid iterator creation.
             for (int i = 0; i < entries.size(); i++) {
                 GridDhtCacheEntry entry = entries.get(i);
@@ -2747,21 +2650,17 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
                     assert writeVal != null || op == DELETE : "null write value found.";
 
-                    boolean primary = !req.fastMap() || ctx.affinity().primaryByPartition(ctx.localNode(),
-                        entry.partition(),
-                        req.topologyVersion());
-
                     Collection<UUID> readers = null;
                     Collection<UUID> filteredReaders = null;
 
                     if (checkReaders) {
                         readers = entry.readers();
-                        filteredReaders = F.view(entry.readers(), F.notEqualTo(node.id()));
+                        filteredReaders = F.view(entry.readers(), F.notEqualTo(nearNode.id()));
                     }
 
                     GridCacheUpdateAtomicResult updRes = entry.innerUpdate(
                         ver,
-                        node.id(),
+                        nearNode.id(),
                         locNodeId,
                         op,
                         writeVal,
@@ -2773,11 +2672,11 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                         expiry,
                         /*event*/true,
                         /*metrics*/true,
-                        primary,
-                        ctx.config().getAtomicWriteOrderMode() == CLOCK, // Check version in CLOCK mode on primary node.
+                        /*primary*/true,
+                        /*verCheck*/false,
                         topVer,
                         null,
-                        replicate ? primary ? DR_PRIMARY : DR_BACKUP : DR_NONE,
+                        replicate ? DR_PRIMARY : DR_NONE,
                         CU.TTL_NOT_CHANGED,
                         CU.EXPIRE_TIME_CALCULATE,
                         null,
@@ -2811,30 +2710,25 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
                     batchRes.addDeleted(entry, updRes, entries);
 
-                    if (dhtFut == null && !F.isEmpty(filteredReaders)) {
-                        dhtFut = createDhtFuture(ver, req, res, completionCb, true);
-
-                        batchRes.readersOnly(true);
-                    }
-
                     if (dhtFut != null) {
                         EntryProcessor<Object, Object, Object> entryProcessor =
                             entryProcessorMap == null ? null : entryProcessorMap.get(entry.key());
 
-                        if (!batchRes.readersOnly()) {
-                            dhtFut.addWriteEntry(entry,
-                                writeVal,
-                                entryProcessor,
-                                updRes.newTtl(),
-                                CU.EXPIRE_TIME_CALCULATE,
-                                null,
-                                sndPrevVal,
-                                updRes.oldValue(),
-                                updRes.updateCounter());
-                        }
+                        dhtFut.addWriteEntry(
+                            affAssignment,
+                            entry,
+                            writeVal,
+                            entryProcessor,
+                            updRes.newTtl(),
+                            CU.EXPIRE_TIME_CALCULATE,
+                            null,
+                            sndPrevVal,
+                            updRes.oldValue(),
+                            updRes.updateCounter());
 
                         if (!F.isEmpty(filteredReaders))
-                            dhtFut.addNearWriteEntries(filteredReaders,
+                            dhtFut.addNearWriteEntries(
+                                filteredReaders,
                                 entry,
                                 writeVal,
                                 entryProcessor,
@@ -2843,30 +2737,26 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                     }
 
                     if (hasNear) {
-                        if (primary) {
-                            if (!ctx.affinity().partitionBelongs(node, entry.partition(), topVer)) {
-                                int idx = firstEntryIdx + i;
-
-                                if (req.operation() == TRANSFORM) {
-                                    res.addNearValue(idx,
-                                        writeVal,
-                                        updRes.newTtl(),
-                                        CU.EXPIRE_TIME_CALCULATE);
-                                }
-                                else
-                                    res.addNearTtl(idx, updRes.newTtl(), CU.EXPIRE_TIME_CALCULATE);
+                        if (!ctx.affinity().partitionBelongs(nearNode, entry.partition(), topVer)) {
+                            int idx = firstEntryIdx + i;
 
-                                if (writeVal != null || entry.hasValue()) {
-                                    IgniteInternalFuture<Boolean> f = entry.addReader(node.id(), req.messageId(), topVer);
-
-                                    assert f == null : f;
-                                }
+                            if (req.operation() == TRANSFORM) {
+                                res.addNearValue(idx,
+                                    writeVal,
+                                    updRes.newTtl(),
+                                    CU.EXPIRE_TIME_CALCULATE);
                             }
-                            else if (readers.contains(node.id())) // Reader became primary or backup.
-                                entry.removeReader(node.id(), req.messageId());
                             else
-                                res.addSkippedIndex(firstEntryIdx + i);
+                                res.addNearTtl(idx, updRes.newTtl(), CU.EXPIRE_TIME_CALCULATE);
+
+                            if (writeVal != null || entry.hasValue()) {
+                                IgniteInternalFuture<Boolean> f = entry.addReader(nearNode.id(), req.messageId(), topVer);
+
+                                assert f == null : f;
+                            }
                         }
+                        else if (readers.contains(nearNode.id())) // Reader became primary or backup.
+                            entry.removeReader(nearNode.id(), req.messageId());
                         else
                             res.addSkippedIndex(firstEntryIdx + i);
                     }
@@ -2879,7 +2769,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             }
         }
         catch (IgniteCheckedException e) {
-            res.addFailedKeys(putMap != null ? putMap.keySet() : rmvKeys, e, ctx);
+            res.addFailedKeys(putMap != null ? putMap.keySet() : rmvKeys, e);
         }
 
         if (storeErr != null) {
@@ -2888,7 +2778,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             for (Object failedKey : storeErr.failedKeys())
                 failed.add(ctx.toCacheKeyObject(failedKey));
 
-            res.addFailedKeys(failed, storeErr.getCause(), ctx);
+            res.addFailedKeys(failed, storeErr.getCause());
         }
 
         return dhtFut;
@@ -2910,23 +2800,14 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             KeyCacheObject key = req.key(0);
 
             while (true) {
-                try {
-                    GridDhtCacheEntry entry = entryExx(key, topVer);
+                GridDhtCacheEntry entry = entryExx(key, topVer);
 
-                    GridUnsafe.monitorEnter(entry);
+                GridUnsafe.monitorEnter(entry);
 
-                    if (entry.obsolete())
-                        GridUnsafe.monitorExit(entry);
-                    else
-                        return Collections.singletonList(entry);
-                }
-                catch (GridDhtInvalidPartitionException e) {
-                    // Ignore invalid partition exception in CLOCK ordering mode.
-                    if (ctx.config().getAtomicWriteOrderMode() == CLOCK)
-                        return Collections.singletonList(null);
-                    else
-                        throw e;
-                }
+                if (entry.obsolete())
+                    GridUnsafe.monitorExit(entry);
+                else
+                    return Collections.singletonList(entry);
             }
         }
         else {
@@ -2934,18 +2815,9 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
             while (true) {
                 for (int i = 0; i < req.size(); i++) {
-                    try {
-                        GridDhtCacheEntry entry = entryExx(req.key(i), topVer);
+                    GridDhtCacheEntry entry = entryExx(req.key(i), topVer);
 
-                        locked.add(entry);
-                    }
-                    catch (GridDhtInvalidPartitionException e) {
-                        // Ignore invalid partition exception in CLOCK ordering mode.
-                        if (ctx.config().getAtomicWriteOrderMode() == CLOCK)
-                            locked.add(null);
-                        else
-                            throw e;
-                    }
+                    locked.add(entry);
                 }
 
                 boolean retry = false;
@@ -3055,7 +2927,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     /**
      * @param req Request to remap.
      */
-    private void remapToNewPrimary(GridNearAtomicAbstractUpdateRequest req) {
+    void remapToNewPrimary(GridNearAtomicAbstractUpdateRequest req) {
         assert req.writeSynchronizationMode() == FULL_ASYNC : req;
 
         if (log.isDebugEnabled())
@@ -3098,7 +2970,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             drPutVals = null;
         }
 
-        final GridNearAtomicUpdateFuture updateFut = new GridNearAtomicUpdateFuture(
+        GridNearAtomicUpdateFuture updateFut = new GridNearAtomicUpdateFuture(
             ctx,
             this,
             ctx.config().getWriteSynchronizationMode(),
@@ -3127,43 +2999,16 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      *
      * @param writeVer Write version.
      * @param updateReq Update request.
-     * @param updateRes Update response.
-     * @param completionCb Completion callback to invoke when future is completed.
-     * @param force If {@code true} then creates future without optimizations checks.
-     * @return Backup update future or {@code null} if there are no backups.
+     * @return Backup update future.
      */
-    @Nullable private GridDhtAtomicAbstractUpdateFuture createDhtFuture(
+    private GridDhtAtomicAbstractUpdateFuture createDhtFuture(
         GridCacheVersion writeVer,
-        GridNearAtomicAbstractUpdateRequest updateReq,
-        GridNearAtomicUpdateResponse updateRes,
-        CI2<GridNearAtomicAbstractUpdateRequest, GridNearAtomicUpdateResponse> completionCb,
-        boolean force
+        GridNearAtomicAbstractUpdateRequest updateReq
     ) {
-        if (!force) {
-            if (updateReq.fastMap())
-                return null;
-
-            AffinityTopologyVersion topVer = updateReq.topologyVersion();
-
-            Collection<ClusterNode> nodes = ctx.kernalContext().discovery().cacheAffinityNodes(ctx.cacheId(), topVer);
-
-            // We are on primary node for some key.
-            assert !nodes.isEmpty() : "Failed to find affinity nodes [name=" + name() + ", topVer=" + topVer +
-                ctx.kernalContext().discovery().discoCache(topVer) + ']';
-
-            if (nodes.size() == 1) {
-                if (log.isDebugEnabled())
-                    log.debug("Partitioned cache topology has only one node, will not create DHT atomic update future " +
-                        "[topVer=" + topVer + ", updateReq=" + updateReq + ']');
-
-                return null;
-            }
-        }
-
         if (updateReq.size() == 1)
-            return new GridDhtAtomicSingleUpdateFuture(ctx, completionCb, writeVer, updateReq, updateRes);
+            return new GridDhtAtomicSingleUpdateFuture(ctx, writeVer, updateReq);
         else
-            return new GridDhtAtomicUpdateFuture(ctx, completionCb, writeVer, updateReq, updateRes);
+            return new GridDhtAtomicUpdateFuture(ctx, writeVer, updateReq);
     }
 
     /**
@@ -3172,13 +3017,10 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      */
     private void processNearAtomicUpdateRequest(UUID nodeId, GridNearAtomicAbstractUpdateRequest req) {
         if (msgLog.isDebugEnabled()) {
-            msgLog.debug("Received near atomic update request [futId=" + req.futureVersion() +
-                ", writeVer=" + req.updateVersion() +
+            msgLog.debug("Received near atomic update request [futId=" + req.futureId() +
                 ", node=" + nodeId + ']');
         }
 
-        req.nodeId(ctx.localNodeId());
-
         updateAllAsyncInternal(nodeId, req, updateReplyClos);
     }
 
@@ -3189,20 +3031,41 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @SuppressWarnings("unchecked")
     private void processNearAtomicUpdateResponse(UUID nodeId, GridNearAtomicUpdateResponse res) {
         if (msgLog.isDebugEnabled())
-            msgLog.debug("Received near atomic update response " +
-                "[futId=" + res.futureVersion() + ", node=" + nodeId + ']');
+            msgLog.debug("Received near atomic update response [futId" + res.futureId() + ", node=" + nodeId + ']');
 
         res.nodeId(ctx.localNodeId());
 
         GridNearAtomicAbstractUpdateFuture fut =
-            (GridNearAtomicAbstractUpdateFuture)ctx.mvcc().atomicFuture(res.futureVersion());
+            (GridNearAtomicAbstractUpdateFuture)ctx.mvcc().atomicFuture(res.futureId());
 
         if (fut != null)
-            fut.onResult(nodeId, res, false);
-
+            fut.onPrimaryResponse(nodeId, res, false);
         else
             U.warn(msgLog, "Failed to find near update future for update response (will ignore) " +
-                "[futId" + res.futureVersion() + ", node=" + nodeId + ", res=" + res + ']');
+                "[futId=" + res.futureId() + ", node=" + nodeId + ", res=" + res + ']');
+    }
+
+    /**
+     * @param nodeId Node ID.
+     * @param checkReq Request.
+     */
+    private void processCheckUpdateRequest(UUID nodeId, GridNearAtomicCheckUpdateRequest checkReq) {
+        /*
+         * Message is processed in the same stripe, so primary already processed update request. It is possible
+         * response was not sent if operation result was empty. Near node will get original response or this one.
+         */
+        GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(ctx.cacheId(),
+            nodeId,
+            checkReq.futureId(),
+            checkReq.partition(),
+            false,
+            false);
+
+        GridCacheReturn ret = new GridCacheReturn(false, true);
+
+        res.returnValue(ret);
+
+        sendNearUpdateReply(nodeId, res);
     }
 
     /**
@@ -3210,20 +3073,28 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @param req Dht atomic update request.
      */
     private void processDhtAtomicUpdateRequest(UUID nodeId, GridDhtAtomicAbstractUpdateRequest req) {
+        assert Thread.currentThread().getName().startsWith("sys-stripe-") : Thread.currentThread().getName();
+
         if (msgLog.isDebugEnabled()) {
-            msgLog.debug("Received DHT atomic update request [futId=" + req.futureVersion() +
+            msgLog.debug("Received DHT atomic update request [futId=" + req.futureId() +
                 ", writeVer=" + req.writeVersion() + ", node=" + nodeId + ']');
         }
 
+        assert req.partition() >= 0 : req;
+
         GridCacheVersion ver = req.writeVersion();
 
-        // Always send update reply.
-        GridDhtAtomicUpdateResponse res = new GridDhtAtomicUpdateResponse(ctx.cacheId(), req.futureVersion(),
-            ctx.deploymentEnabled());
+        GridDhtAtomicNearResponse nearRes = null;
 
-        res.partition(req.partition());
+        if (req.nearNodeId() != null) {
+            nearRes = new GridDhtAtomicNearResponse(ctx.cacheId(),
+                req.partition(),
+                req.nearFutureId(),
+                nodeId,
+                req.flags());
+        }
 
-        Boolean replicate = ctx.isDrEnabled();
+        boolean replicate = ctx.isDrEnabled();
 
         boolean intercept = req.forceTransformBackups() && ctx.config().getInterceptor() != null;
 
@@ -3305,48 +3176,208 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                 // Ignore.
             }
             catch (IgniteCheckedException e) {
-                res.addFailedKey(key, new IgniteCheckedException("Failed to update key on backup node: " + key, e));
+                IgniteCheckedException err =
+                    new IgniteCheckedException("Failed to update key on backup node: " + key, e);
+
+                if (nearRes != null)
+                    nearRes.addFailedKey(key, err);
+
+                U.error(log, "Failed to update key on backup node: " + key, e);
+            }
+        }
+
+        GridDhtAtomicUpdateResponse dhtRes = null;
+
+        if (isNearEnabled(cacheCfg)) {
+            List<KeyCacheObject> nearEvicted =
+                ((GridNearAtomicCache<K, V>)near()).processDhtAtomicUpdateRequest(nodeId, req, nearRes);
+
+            if (nearEvicted != null) {
+                dhtRes = new GridDhtAtomicUpdateResponse(ctx.cacheId(),
+                    req.partition(),
+                    req.futureId(),
+                    ctx.deploymentEnabled());
+
+                dhtRes.nearEvicted(nearEvicted);
             }
         }
 
-        if (isNearEnabled(cacheCfg))
-            ((GridNearAtomicCache<K, V>)near()).processDhtAtomicUpdateRequest(nodeId, req, res);
+        if (nearRes != null)
+            sendDhtNearResponse(req, nearRes);
 
+        if (dhtRes == null && req.replyWithoutDelay()) {
+            dhtRes = new GridDhtAtomicUpdateResponse(ctx.cacheId(),
+                req.partition(),
+                req.futureId(),
+                ctx.deploymentEnabled());
+        }
+
+        if (dhtRes != null)
+            sendDhtPrimaryResponse(nodeId, req, dhtRes);
+        else
+            sendDeferredUpdateResponse(req.partition(), nodeId, req.futureId());
+    }
+
+    /**
+     * @param nodeId Primary node ID.
+     * @param req Request.
+     * @param dhtRes Response to send.
+     */
+    private void sendDhtPrimaryResponse(UUID nodeId,
+        GridDhtAtomicAbstractUpdateRequest req,
+        GridDhtAtomicUpdateResponse dhtRes) {
         try {
-            if (res.failedKeys() != null || res.nearEvicted() != null || req.writeSynchronizationMode() == FULL_SYNC) {
-                ctx.io().send(nodeId, res, ctx.ioPolicy());
+            ctx.io().send(nodeId, dhtRes, ctx.ioPolicy());
 
-                if (msgLog.isDebugEnabled()) {
-                    msgLog.debug("Sent DHT atomic update response [futId=" + req.futureVersion() +
-                        ", writeVer=" + req.writeVersion() + ", node=" + nodeId + ']');
-                }
+            if (msgLog.isDebugEnabled()) {
+                msgLog.debug("Sent DHT response [futId=" + req.futureId() +
+                    ", nearFutId=" + req.nearFutureId() +
+                    ", writeVer=" + req.writeVersion() +
+                    ", node=" + nodeId + ']');
             }
-            else {
-                if (msgLog.isDebugEnabled()) {
-                    msgLog.debug("Will send deferred DHT atomic update response [futId=" + req.futureVersion() +
-                        ", writeVer=" + req.writeVersion() + ", node=" + nodeId + ']');
-                }
+        }
+        catch (ClusterTopologyCheckedException ignored) {
+            U.warn(msgLog, "Failed to send DHT response, node left [futId=" + req.futureId() +
+                ", nearFutId=" + req.nearFutureId() +
+                ", node=" + nodeId + ']');
+        }
+        catch (IgniteCheckedException e) {
+            U.error(msgLog, "Failed to send DHT near response [futId=" + req.futureId() +
+                ", nearFutId=" + req.nearFutureId() +
+                ", node=" + nodeId +
+                ", res=" + dhtRes + ']', e);
+        }
+    }
+
+    /**
+     * @param part Partition.
+     * @param primaryId Primary ID.
+     * @param futId Future ID.
+     */
+    private void sendDeferredUpdateResponse(int part, UUID primaryId, long futId) {
+        Map<UUID, GridDhtAtomicDeferredUpdateResponse> resMap = defRes.get();
+
+        GridDhtAtomicDeferredUpdateResponse msg = resMap.get(primaryId);
+
+        if (msg == null) {
+            msg = new GridDhtAtomicDeferredUpdateResponse(ctx.cacheId(),
+                new GridLongList(DEFERRED_UPDATE_RESPONSE_BUFFER_SIZE));
+
+            if (DEFERRED_UPDATE_RESPONSE_TIMEOUT > 0) {
+                GridTimeoutObject timeoutSnd = new DeferredUpdateTimeout(part, primaryId);
+
+                msg.timeoutSender(timeoutSnd);
+
+                ctx.time().addTimeoutObject(timeoutSnd);
+            }
+
+            resMap.put(primaryId, msg);
+        }
+
+        GridLongList futIds = msg.futureIds();
+
+        assert futIds.size() < DEFERRED_UPDATE_RESPONSE_BUFFER_SIZE : futIds.size();
+
+        futIds.add(futId);
+
+        if (futIds.size() >= DEFERRED_UPDATE_RESPONSE_BUFFER_SIZE) {
+            resMap.remove(primaryId);
+
+            sendDeferredUpdateResponse(primaryId, msg);
+        }
+    }
+
+    /**
+     * @param primaryId Primary ID.
+     * @param msg Message.
+     */
+    private void sendDeferredUpdateResponse(UUID primaryId, GridDhtAtomicDeferredUpdateResponse msg) {
+        try {
+            GridTimeoutObject timeoutSnd = msg.timeoutSender();
+
+            if (timeoutSnd != null)
+                ctx.time().removeTimeoutObject(timeoutSnd);
+
+            ctx.io().send(primaryId, msg, ctx.ioPolicy());
+
+            if (msgLog.isDebugEnabled()) {
+                msgLog.debug("Sent deferred DHT update response [futIds=" + msg.futureIds() +
+                    ", node=" + primaryId + ']');
+            }
+        }
+        catch (ClusterTopologyCheckedException ignored) {
+            if (msgLog.isDebugEnabled()) {
+                msgLog.debug("Failed to send deferred DHT update response, node left [" +
+                    "futIds=" + msg.futureIds() + ", node=" + primaryId + ']');
+            }
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send deferred DHT update response to remote node [" +
+                "futIds=" + msg.futureIds() + ", node=" + primaryId + ']', e);
+        }
+    }
+
+    /**
+     * @param req Request.
+     * @param nearRes Response to send.
+     */
+    private void sendDhtNearResponse(final GridDhtAtomicAbstractUpdateRequest req, GridDhtAtomicNearResponse nearRes) {
+        try {
+            ClusterNode node = ctx.discovery().node(req.nearNodeId());
+
+            if (node == null)
+                throw new ClusterTopologyCheckedException("Node failed: " + req.nearNodeId());
 
-                // No failed keys and sync mode is not FULL_SYNC, thus sending deferred response.
-                sendDeferredUpdateResponse(nodeId, req.futureVersion());
+            if (node.isLocal())
+                processDhtAtomicNearResponse(node.id(), nearRes);
+            else
+                ctx.io().send(node, nearRes, ctx.ioPolicy());
+
+            if (msgLog.isDebugEnabled()) {
+                msgLog.debug("Sent DHT near response [futId=" + req.futureId() +
+                    ", nearFutId=" + req.nearFutureId() +
+                    ", writeVer=" + req.writeVersion() +
+                    ", node=" + req.nearNodeId() + ']');
             }
         }
         catch (ClusterTopologyCheckedException ignored) {
-            U.warn(msgLog, "Failed to send DHT atomic update response, node left [futId=" + req.futureVersion() +
-                ", node=" + req.nodeId() + ']');
+            if (msgLog.isDebugEnabled()) {
+                msgLog.debug("Failed to send DHT near response, node left [futId=" + req.futureId() +
+                    ", nearFutId=" + req.nearFutureId() +
+                    ", node=" + req.nearNodeId() + ']');
+            }
         }
         catch (IgniteCheckedException e) {
-            U.error(msgLog, "Failed to send DHT atomic update response [futId=" + req.futureVersion() +
-                ", node=" + nodeId +  ", res=" + res + ']', e);
+            U.error(msgLog, "Failed to send DHT near response [futId=" + req.futureId() +
+                ", nearFutId=" + req.nearFutureId() +
+                ", node=" + req.nearNodeId() +
+                ", res=" + nearRes + ']', e);
         }
     }
 
     /**
-     * @param nodeId Node ID to send message to.
-     * @param ver Version to ack.
+     * @param nodeId Node ID.
+     * @param res Response.
      */
-    private void sendDeferredUpdateResponse(UUID nodeId, GridCacheVersion ver) {
-        deferredUpdateMsgSnd.sendDeferredAckMessage(nodeId, ver);
+    private void processDhtAtomicNearResponse(UUID nodeId, GridDhtAtomicNearResponse res) {
+        GridNearAtomicAbstractUpdateFuture updateFut =
+            (GridNearAtomicAbstractUpdateFuture)ctx.mvcc().atomicFuture(res.futureId());
+
+        if (updateFut != null) {
+            if (msgLog.isDebugEnabled()) {
+                msgLog.debug("Received DHT atomic near response [futId=" + res.futureId() +
+                    ", node=" + nodeId + ']');
+            }
+
+            updateFut.onDhtResponse(nodeId, res);
+        }
+        else {
+            if (msgLog.isDebugEnabled()) {
+                msgLog.debug("Failed to find future for DHT atomic near response [futId=" + res.futureId() +
+                    ", node=" + nodeId +
+                    ", res=" + res + ']');
+            }
+        }
     }
 
     /**
@@ -3355,18 +3386,19 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      */
     @SuppressWarnings("unchecked")
     private void processDhtAtomicUpdateResponse(UUID nodeId, GridDhtAtomicUpdateResponse res) {
-        GridDhtAtomicAbstractUpdateFuture updateFut = (GridDhtAtomicAbstractUpdateFuture)ctx.mvcc().atomicFuture(res.futureVersion());
+        GridDhtAtomicAbstractUpdateFuture updateFut =
+            (GridDhtAtomicAbstractUpdateFuture)ctx.mvcc().atomicFuture(res.futureId());
 
         if (updateFut != null) {
             if (msgLog.isDebugEnabled()) {
-                msgLog.debug("Received DHT atomic update response [futId=" + res.futureVersion() +
-                    ", writeVer=" + updateFut.writeVersion() + ", node=" + nodeId + ']');
+                msgLog.debug("Received DHT atomic update response [futId=" + res.futureId() +
+                        ", writeVer=" + updateFut.writeVersion() + ", node=" + nodeId + ']');
             }
 
-            updateFut.onResult(nodeId, res);
+            updateFut.onDhtResponse(nodeId, res);
         }
         else {
-            U.warn(msgLog, "Failed to find DHT update future for update response [futId=" + res.futureVersion() +
+            U.warn(msgLog, "Failed to find DHT update future for update response [futId=" + res.futureId() +
                 ", node=" + nodeId + ", res=" + res + ']');
         }
     }
@@ -3377,19 +3409,25 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      */
     @SuppressWarnings("unchecked")
     private void processDhtAtomicDeferredUpdateResponse(UUID nodeId, GridDhtAtomicDeferredUpdateResponse res) {
-        for (GridCacheVersion ver : res.futureVersions()) {
-            GridDhtAtomicAbstractUpdateFuture updateFut = (GridDhtAtomicAbstractUpdateFuture)ctx.mvcc().atomicFuture(ver);
+        GridLongList futIds = res.futureIds();
+
+        assert futIds != null && futIds.size() > 0 : futIds;
+
+        for (int i = 0; i < futIds.size(); i++) {
+            Long id = futIds.get(i);
+
+            GridDhtAtomicAbstractUpdateFuture updateFut = (GridDhtAtomicAbstractUpdateFuture)ctx.mvcc().atomicFuture(id);
 
             if (updateFut != null) {
                 if (msgLog.isDebugEnabled()) {
-                    msgLog.debug("Received DHT atomic deferred update response [futId=" + ver +
+                    msgLog.debug("Received DHT atomic deferred update response [futId=" + id +
                         ", writeVer=" + res + ", node=" + nodeId + ']');
                 }
 
-                updateFut.onResult(nodeId);
+                updateFut.onDeferredResponse(nodeId);
             }
             else {
-                U.warn(msgLog, "Failed to find DHT update future for deferred update response [futId=" + ver +
+                U.warn(msgLog, "Failed to find DHT update future for deferred update response [futId=" + id +
                     ", nodeId=" + nodeId + ", res=" + res + ']');
             }
         }
@@ -3404,16 +3442,16 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             ctx.io().send(nodeId, res, ctx.ioPolicy());
 
             if (msgLog.isDebugEnabled())
-                msgLog.debug("Sent near update response [futId=" + res.futureVersion() + ", node=" + nodeId + ']');
+                msgLog.debug("Sent near update response [futId=" + res.futureId() + ", node=" + nodeId + ']');
         }
         catch (ClusterTopologyCheckedException ignored) {
             if (msgLog.isDebugEnabled()) {
-                msgLog.debug("Failed to send near update response [futId=" + res.futureVersion() +
+                msgLog.debug("Failed to send near update response [futId=" + res.futureId() +
                     ", node=" + nodeId + ']');
             }
         }
         catch (IgniteCheckedException e) {
-            U.error(msgLog, "Failed to send near update response [futId=" + res.futureVersion() +
+            U.error(msgLog, "Failed to send near update response [futId=" + res.futureId() +
                 ", node=" + nodeId + ", res=" + res + ']', e);
         }
     }
@@ -3482,9 +3520,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         private GridDhtAtomicAbstractUpdateFuture dhtFut;
 
         /** */
-        private boolean readersOnly;
-
-        /** */
         private GridCacheReturn invokeRes;
 
         /**
@@ -3537,20 +3572,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         private void dhtFuture(@Nullable GridDhtAtomicAbstractUpdateFuture dhtFut) {
             this.dhtFut = dhtFut;
         }
-
-        /**
-         * @return {@code True} if only readers (not backups) should be updated.
-         */
-        private boolean readersOnly() {
-            return readersOnly;
-        }
-
-        /**
-         * @param readersOnly {@code True} if only readers (not backups) should be updated.
-         */
-        private void readersOnly(boolean readersOnly) {
-            this.readersOnly = readersOnly;
-        }
     }
 
     /**
@@ -3569,4 +3590,71 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             return Collections.emptyList();
         }
     }
+
+    /**
+     *
+     */
+    interface UpdateReplyClosure extends CI2<GridNearAtomicAbstractUpdateRequest, GridNearAtomicUpdateResponse> {
+        // No-op.
+    }
+
+    /**
+     *
+     */
+    private class DeferredUpdateTimeout implements GridTimeoutObject, Runnable {
+        /** */
+        private final int part;
+
+        /** */
+        private final UUID primaryId;
+
+        /** */
+        private final IgniteUuid id;
+
+        /** */
+        private final long endTime;
+
+        /**
+         * @param part Partition.
+         * @param primaryId Primary ID.
+         */
+        DeferredUpdateTimeout(int part, UUID primaryId) {
+            this.part = part;
+            this.primaryId = primaryId;
+
+            endTime = U.currentTimeMillis() + DEFERRED_UPDATE_RESPONSE_TIMEOUT;
+
+            id = IgniteUuid.fromUuid(primaryId);
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgniteUuid timeoutId() {
+            return id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public long endTime() {
+            return endTime;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            Map<UUID, GridDhtAtomicDeferredUpdateResponse> resMap = defRes.get();
+
+            GridDhtAtomicDeferredUpdateResponse msg = resMap.get(primaryId);
+
+            if (msg != null && msg.timeoutSender() == this) {
+                msg.timeoutSender(null);
+
+                resMap.remove(primaryId);
+
+                sendDeferredUpdateResponse(primaryId, msg);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onTimeout() {
+            ctx.kernalContext().getStripedExecutorService().execute(part, this);
+        }
+    }
 }


[8/8] ignite git commit: ignite-4705 Atomic cache protocol change: notify client node from backups

Posted by sb...@apache.org.
ignite-4705 Atomic cache protocol change: notify client node from backups


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

Branch: refs/heads/ignite-2.0
Commit: cbc472fe7f058db42ce49652c85981c7b797d229
Parents: f59f46d
Author: sboikov <sb...@gridgain.com>
Authored: Mon Mar 13 18:07:20 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Mar 13 18:08:50 2017 +0300

----------------------------------------------------------------------
 .../rest/protocols/tcp/MockNioSession.java      |   5 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   5 +-
 .../internal/binary/BinaryObjectImpl.java       |  43 +-
 .../connection/GridClientNioTcpConnection.java  |   2 +-
 .../managers/communication/GridIoManager.java   |   5 +-
 .../communication/GridIoMessageFactory.java     |  20 +-
 .../processors/cache/CacheObjectContext.java    |   3 +-
 .../processors/cache/GridCacheAtomicFuture.java |   5 +-
 .../processors/cache/GridCacheIoManager.java    |  83 +-
 .../processors/cache/GridCacheMapEntry.java     |  12 +-
 .../processors/cache/GridCacheMessage.java      |  17 +-
 .../processors/cache/GridCacheMvccManager.java  |  48 +-
 .../processors/cache/GridCacheProcessor.java    |   1 -
 .../processors/cache/GridCacheReturn.java       |   6 +-
 .../cache/GridDeferredAckMessageSender.java     |  17 +-
 .../processors/cache/KeyCacheObjectImpl.java    |  65 +-
 .../dht/GridClientPartitionTopology.java        |   8 +
 .../dht/GridDhtPartitionTopology.java           |   9 +
 .../dht/GridDhtPartitionTopologyImpl.java       |  23 +-
 .../GridDhtAtomicAbstractUpdateFuture.java      | 298 +++---
 .../GridDhtAtomicAbstractUpdateRequest.java     | 392 +++++++-
 .../dht/atomic/GridDhtAtomicCache.java          | 896 +++++++++--------
 .../GridDhtAtomicDeferredUpdateResponse.java    |  68 +-
 .../dht/atomic/GridDhtAtomicNearResponse.java   | 314 ++++++
 .../atomic/GridDhtAtomicSingleUpdateFuture.java | 101 +-
 .../GridDhtAtomicSingleUpdateRequest.java       | 277 +-----
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |  89 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  | 325 ++-----
 .../dht/atomic/GridDhtAtomicUpdateResponse.java | 124 +--
 ...idNearAtomicAbstractSingleUpdateRequest.java | 481 +---------
 .../GridNearAtomicAbstractUpdateFuture.java     | 468 +++++++--
 .../GridNearAtomicAbstractUpdateRequest.java    | 480 ++++++++-
 .../GridNearAtomicCheckUpdateRequest.java       | 175 ++++
 .../atomic/GridNearAtomicFullUpdateRequest.java | 487 +---------
 ...GridNearAtomicSingleUpdateFilterRequest.java |  23 +-
 .../GridNearAtomicSingleUpdateFuture.java       | 617 ++++++------
 ...GridNearAtomicSingleUpdateInvokeRequest.java |  37 +-
 .../GridNearAtomicSingleUpdateRequest.java      |  65 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  | 962 +++++++++++--------
 .../atomic/GridNearAtomicUpdateResponse.java    | 192 ++--
 .../distributed/dht/atomic/UpdateErrors.java    | 222 +++++
 .../distributed/near/GridNearAtomicCache.java   |  27 +-
 .../continuous/CacheContinuousQueryHandler.java |   2 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |   5 +-
 .../IgniteCacheObjectProcessorImpl.java         |  18 +-
 .../ignite/internal/util/StripedExecutor.java   |   8 +-
 .../util/future/GridCompoundFuture.java         |  11 +-
 .../internal/util/ipc/IpcToNioAdapter.java      |   7 +-
 .../nio/GridConnectionBytesVerifyFilter.java    |   7 +-
 .../util/nio/GridNioAsyncNotifyFilter.java      |   7 +-
 .../internal/util/nio/GridNioCodecFilter.java   |   9 +-
 .../util/nio/GridNioEmbeddedFuture.java         |   7 +
 .../ignite/internal/util/nio/GridNioFilter.java |  12 +-
 .../internal/util/nio/GridNioFilterAdapter.java |   7 +-
 .../internal/util/nio/GridNioFilterChain.java   |  15 +-
 .../util/nio/GridNioFinishedFuture.java         |   5 -
 .../ignite/internal/util/nio/GridNioFuture.java |   7 -
 .../internal/util/nio/GridNioFutureImpl.java    |  18 +-
 .../ignite/internal/util/nio/GridNioServer.java |  83 +-
 .../internal/util/nio/GridNioSession.java       |   7 +-
 .../internal/util/nio/GridNioSessionImpl.java   |   9 +-
 .../util/nio/GridNioSessionMetaKey.java         |   5 +-
 .../util/nio/GridShmemCommunicationClient.java  |   6 +-
 .../util/nio/GridTcpNioCommunicationClient.java |  13 +-
 .../internal/util/nio/SessionWriteRequest.java  |   7 -
 .../internal/util/nio/ssl/GridNioSslFilter.java |  12 +-
 .../util/nio/ssl/GridNioSslHandler.java         |  29 +-
 .../communication/tcp/TcpCommunicationSpi.java  |   2 +-
 .../org/apache/ignite/thread/IgniteThread.java  |  41 +-
 .../ignite/thread/IgniteThreadFactory.java      |   2 +-
 .../GridCommunicationSendMessageSelfTest.java   |   2 +-
 .../cache/CacheRebalancingSelfTest.java         |  16 +-
 .../GridCacheAbstractFailoverSelfTest.java      |   2 -
 .../GridCacheAtomicMessageCountSelfTest.java    |  22 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |   1 +
 ...niteCacheClientNodeChangingTopologyTest.java |   7 -
 .../IgniteCacheMessageRecoveryAbstractTest.java |   2 +-
 .../dht/GridCacheAtomicNearCacheSelfTest.java   |  23 +-
 .../IgniteCachePutRetryAbstractSelfTest.java    |  37 +-
 ...gniteCachePutRetryTransactionalSelfTest.java |   2 +-
 .../atomic/IgniteCacheAtomicProtocolTest.java   | 883 +++++++++++++++++
 ...erNoStripedPoolMultiNodeFullApiSelfTest.java |  35 -
 .../near/GridCacheNearReadersSelfTest.java      |  17 +-
 ...edNoStripedPoolMultiNodeFullApiSelfTest.java |  35 -
 ...eContinuousQueryAsyncFilterListenerTest.java |   2 +-
 ...ContinuousQueryFailoverAbstractSelfTest.java |  31 +-
 ...eCacheContinuousQueryImmutableEntryTest.java |   2 +-
 .../nio/GridNioEmbeddedFutureSelfTest.java      |   2 +-
 .../util/future/nio/GridNioFutureSelfTest.java  |  25 +-
 .../nio/impl/GridNioFilterChainSelfTest.java    |  12 +-
 .../file/GridFileSwapSpaceSpiSelfTest.java      |   2 +-
 .../IgniteCacheFullApiSelfTestSuite.java        |   8 +-
 .../testsuites/IgniteCacheTestSuite5.java       |   3 +
 .../HadoopExternalCommunication.java            |   9 +-
 .../communication/HadoopIpcToNioAdapter.java    |   7 +-
 .../communication/HadoopMarshallerFilter.java   |  10 +-
 .../cache/IgniteGetAndPutBenchmark.java         |   2 +-
 .../cache/IgniteGetAndPutTxBenchmark.java       |   2 +-
 98 files changed, 5462 insertions(+), 3597 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/MockNioSession.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/MockNioSession.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/MockNioSession.java
index 9bc4e7f..9d1755f 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/MockNioSession.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/MockNioSession.java
@@ -19,11 +19,13 @@ package org.apache.ignite.internal.processors.rest.protocols.tcp;
 
 import java.net.InetSocketAddress;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter;
 import org.apache.ignite.internal.util.nio.GridNioFinishedFuture;
 import org.apache.ignite.internal.util.nio.GridNioFuture;
 import org.apache.ignite.internal.util.nio.GridNioRecoveryDescriptor;
 import org.apache.ignite.internal.util.nio.GridNioSession;
+import org.apache.ignite.lang.IgniteInClosure;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -112,7 +114,8 @@ public class MockNioSession extends GridMetadataAwareAdapter implements GridNioS
     }
 
     /** {@inheritDoc} */
-    @Override public void sendNoFuture(Object msg) throws IgniteCheckedException {
+    @Override public void sendNoFuture(Object msg, @Nullable IgniteInClosure<IgniteException> ackC)
+        throws IgniteCheckedException {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 2d35cdb..f6cfe12 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1703,8 +1703,9 @@ public class IgnitionEx {
 
             sysExecSvc.allowCoreThreadTimeOut(true);
 
-            if (cfg.getStripedPoolSize() > 0)
-                stripedExecSvc = new StripedExecutor(cfg.getStripedPoolSize(), cfg.getIgniteInstanceName(), "sys", log);
+            validateThreadPoolSize(cfg.getStripedPoolSize(), "stripedPool");
+
+            stripedExecSvc = new StripedExecutor(cfg.getStripedPoolSize(), cfg.getIgniteInstanceName(), "sys", log);
 
             // Note that since we use 'LinkedBlockingQueue', number of
             // maximum threads has no effect.

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
index 7a81659..6fe1a3b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
@@ -17,6 +17,17 @@
 
 package org.apache.ignite.internal.binary;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Date;
+import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectException;
@@ -33,19 +44,7 @@ import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.jetbrains.annotations.Nullable;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.util.Date;
-import java.util.UUID;
-
-import static java.nio.charset.StandardCharsets.*;
+import static java.nio.charset.StandardCharsets.UTF_8;
 
 /**
  * Binary object implementation.
@@ -74,7 +73,6 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
     private boolean detachAllowed;
 
     /** */
-    @GridDirectTransient
     private int part = -1;
 
     /**
@@ -561,7 +559,6 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
 
         start = in.readInt();
     }
-
     /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
@@ -584,6 +581,12 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
                 writer.incrementState();
 
             case 1:
+                if (!writer.writeInt("part", part))
+                    return false;
+
+                writer.incrementState();
+
+            case 2:
                 if (!writer.writeInt("start", detachAllowed ? 0 : start))
                     return false;
 
@@ -611,6 +614,14 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
                 reader.incrementState();
 
             case 1:
+                part = reader.readInt("part");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 2:
                 start = reader.readInt("start");
 
                 if (!reader.isLastRead())
@@ -620,7 +631,7 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
 
         }
 
-        return true;
+        return reader.afterMessageRead(BinaryObjectImpl.class);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
index 8937504..d3a30fb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
@@ -229,7 +229,7 @@ public class GridClientNioTcpConnection extends GridClientConnection {
             GridNioFuture<?> sslHandshakeFut = null;
 
             if (sslCtx != null) {
-                sslHandshakeFut = new GridNioFutureImpl<>();
+                sslHandshakeFut = new GridNioFutureImpl<>(null);
 
                 meta.put(GridNioSslFilter.HANDSHAKE_FUT_META_KEY, sslHandshakeFut);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index 0c0dbf7..23738d7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -818,10 +818,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
             return;
         }
 
-        if (ctx.config().getStripedPoolSize() > 0 &&
-            plc == GridIoPolicy.SYSTEM_POOL &&
-            msg.partition() != Integer.MIN_VALUE
-            ) {
+        if (plc == GridIoPolicy.SYSTEM_POOL && msg.partition() != Integer.MIN_VALUE) {
             ctx.getStripedExecutorService().execute(msg.partition(), c);
 
             return;

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index 6f95400..0548581 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@ -67,14 +67,17 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrep
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareResponse;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtUnlockRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicDeferredUpdateResponse;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicNearResponse;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicSingleUpdateRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateResponse;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicCheckUpdateRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicFullUpdateRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFilterRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateInvokeRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.UpdateErrors;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysResponse;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage;
@@ -118,11 +121,11 @@ import org.apache.ignite.internal.processors.datastreamer.DataStreamerEntry;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerRequest;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerResponse;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
+import org.apache.ignite.internal.processors.hadoop.shuffle.HadoopDirectShuffleMessage;
 import org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleAck;
 import org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleFinishRequest;
 import org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleFinishResponse;
 import org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleMessage;
-import org.apache.ignite.internal.processors.hadoop.shuffle.HadoopDirectShuffleMessage;
 import org.apache.ignite.internal.processors.igfs.IgfsAckMessage;
 import org.apache.ignite.internal.processors.igfs.IgfsBlockKey;
 import org.apache.ignite.internal.processors.igfs.IgfsBlocksMessage;
@@ -173,6 +176,21 @@ public class GridIoMessageFactory implements MessageFactory {
         Message msg = null;
 
         switch (type) {
+            case -47:
+                msg = new GridNearAtomicCheckUpdateRequest();
+
+                break;
+
+            case -46:
+                msg = new UpdateErrors();
+
+                break;
+
+            case -45:
+                msg = new GridDhtAtomicNearResponse();
+
+                break;
+
             case -44:
                 msg = new TcpCommunicationSpi.HandshakeMessage2();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
index c4203ef..a777ab6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
@@ -33,7 +33,8 @@ import org.apache.ignite.internal.util.typedef.F;
 /**
  *
  */
-@SuppressWarnings("TypeMayBeWeakened") public class CacheObjectContext {
+@SuppressWarnings("TypeMayBeWeakened")
+public class CacheObjectContext {
     /** */
     private GridKernalContext kernalCtx;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicFuture.java
index 3e11d50..8df229e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicFuture.java
@@ -19,16 +19,15 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 
 /**
  * Update future for atomic cache.
  */
 public interface GridCacheAtomicFuture<R> extends GridCacheFuture<R> {
     /**
-     * @return Future version.
+     * @return Future ID.
      */
-    public GridCacheVersion version();
+    public Long id();
 
     /**
      * Gets future that will be completed when it is safe when update is finished on the given version of topology.

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index 1f28201..1cd8fbe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -47,15 +47,18 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrep
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareResponse;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedSingleGetFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateRequest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicNearResponse;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicSingleUpdateRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateResponse;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateRequest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicCheckUpdateRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicFullUpdateRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFilterRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateInvokeRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.UpdateErrors;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysResponse;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetRequest;
@@ -85,6 +88,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiInClosure;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
@@ -213,15 +217,19 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             }
 
             if (fut != null && !fut.isDone()) {
+                Thread curThread = Thread.currentThread();
+
+                final int stripe = curThread instanceof IgniteThread ? ((IgniteThread)curThread).stripe() : -1;
+
                 fut.listen(new CI1<IgniteInternalFuture<?>>() {
                     @Override public void apply(IgniteInternalFuture<?> t) {
-                        cctx.kernalContext().closure().runLocalSafe(new Runnable() {
+                        Runnable c = new Runnable() {
                             @Override public void run() {
                                 IgniteLogger log = cacheMsg.messageLogger(cctx);
 
                                 if (log.isDebugEnabled()) {
                                     StringBuilder msg0 = new StringBuilder("Process cache message after wait for " +
-                                        "affinity topology version [");
+                                            "affinity topology version [");
 
                                     appendMessageInfo(cacheMsg, nodeId, msg0).append(']');
 
@@ -230,7 +238,12 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                                 handleMessage(nodeId, cacheMsg);
                             }
-                        });
+                        };
+
+                        if (stripe >= 0)
+                            cctx.kernalContext().getStripedExecutorService().execute(stripe, c);
+                        else
+                            cctx.kernalContext().closure().runLocalSafe(c);
                     }
                 });
 
@@ -471,15 +484,17 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      * @param cacheMsg Cache message.
      * @return Atomic future ID if applicable for message.
      */
-    @Nullable private GridCacheVersion atomicFututeId(GridCacheMessage cacheMsg) {
+    @Nullable private Long atomicFututeId(GridCacheMessage cacheMsg) {
         if (cacheMsg instanceof GridNearAtomicAbstractUpdateRequest)
-            return ((GridNearAtomicAbstractUpdateRequest)cacheMsg).futureVersion();
+            return ((GridNearAtomicAbstractUpdateRequest)cacheMsg).futureId();
         else if (cacheMsg instanceof GridNearAtomicUpdateResponse)
-            return ((GridNearAtomicUpdateResponse) cacheMsg).futureVersion();
+            return ((GridNearAtomicUpdateResponse) cacheMsg).futureId();
         else if (cacheMsg instanceof GridDhtAtomicAbstractUpdateRequest)
-            return ((GridDhtAtomicAbstractUpdateRequest)cacheMsg).futureVersion();
+            return ((GridDhtAtomicAbstractUpdateRequest)cacheMsg).futureId();
         else if (cacheMsg instanceof GridDhtAtomicUpdateResponse)
-            return ((GridDhtAtomicUpdateResponse) cacheMsg).futureVersion();
+            return ((GridDhtAtomicUpdateResponse) cacheMsg).futureId();
+        else if (cacheMsg instanceof GridNearAtomicCheckUpdateRequest)
+            return ((GridNearAtomicCheckUpdateRequest)cacheMsg).futureId();
 
         return null;
     }
@@ -490,9 +505,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      * @return Atomic future ID if applicable for message.
      */
     @Nullable private GridCacheVersion atomicWriteVersion(GridCacheMessage cacheMsg) {
-        if (cacheMsg instanceof GridNearAtomicAbstractUpdateRequest)
-            return ((GridNearAtomicAbstractUpdateRequest)cacheMsg).updateVersion();
-        else if (cacheMsg instanceof GridDhtAtomicAbstractUpdateRequest)
+        if (cacheMsg instanceof GridDhtAtomicAbstractUpdateRequest)
             return ((GridDhtAtomicAbstractUpdateRequest)cacheMsg).writeVersion();
 
         return null;
@@ -561,12 +574,25 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                 GridDhtAtomicUpdateResponse res = new GridDhtAtomicUpdateResponse(
                     ctx.cacheId(),
-                    req.futureVersion(),
+                    req.partition(),
+                    req.futureId(),
                     ctx.deploymentEnabled());
 
                 res.onError(req.classError());
 
                 sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
+
+                if (req.nearNodeId() != null) {
+                    GridDhtAtomicNearResponse nearRes = new GridDhtAtomicNearResponse(ctx.cacheId(),
+                        req.partition(),
+                        req.nearFutureId(),
+                        nodeId,
+                        req.flags());
+
+                    nearRes.errors(new UpdateErrors(req.classError()));
+
+                    sendResponseOnFailedMessage(req.nearNodeId(), nearRes, cctx, ctx.ioPolicy());
+                }
             }
 
             break;
@@ -577,7 +603,9 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
                 GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(
                     ctx.cacheId(),
                     nodeId,
-                    req.futureVersion(),
+                    req.futureId(),
+                    req.partition(),
+                    false,
                     ctx.deploymentEnabled());
 
                 res.error(req.classError());
@@ -755,7 +783,9 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
                 GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(
                     ctx.cacheId(),
                     nodeId,
-                    req.futureVersion(),
+                    req.futureId(),
+                    req.partition(),
+                    false,
                     ctx.deploymentEnabled());
 
                 res.error(req.classError());
@@ -771,7 +801,9 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
                 GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(
                     ctx.cacheId(),
                     nodeId,
-                    req.futureVersion(),
+                    req.futureId(),
+                    req.partition(),
+                    false,
                     ctx.deploymentEnabled());
 
                 res.error(req.classError());
@@ -787,7 +819,9 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
                 GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(
                     ctx.cacheId(),
                     nodeId,
-                    req.futureVersion(),
+                    req.futureId(),
+                    req.partition(),
+                    false,
                     ctx.deploymentEnabled());
 
                 res.error(req.classError());
@@ -802,12 +836,25 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                 GridDhtAtomicUpdateResponse res = new GridDhtAtomicUpdateResponse(
                     ctx.cacheId(),
-                    req.futureVersion(),
+                    req.partition(),
+                    req.futureId(),
                     ctx.deploymentEnabled());
 
                 res.onError(req.classError());
 
                 sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
+
+                if (req.nearNodeId() != null) {
+                    GridDhtAtomicNearResponse nearRes = new GridDhtAtomicNearResponse(ctx.cacheId(),
+                        req.partition(),
+                        req.nearFutureId(),
+                        nodeId,
+                        req.flags());
+
+                    nearRes.errors(new UpdateErrors(req.classError()));
+
+                    sendResponseOnFailedMessage(req.nearNodeId(), nearRes, cctx, ctx.ioPolicy());
+                }
             }
 
             break;
@@ -894,7 +941,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      */
     @SuppressWarnings("unchecked")
     public void send(ClusterNode node, GridCacheMessage msg, byte plc) throws IgniteCheckedException {
-        assert !node.isLocal();
+        assert !node.isLocal() : node;
 
         if (!onSend(msg, node.id()))
             return;

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 2237e22..54b4ed7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -2170,8 +2170,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                     assert conflictCtx != null;
 
-                    boolean ignoreTime = cctx.config().getAtomicWriteOrderMode() == CacheAtomicWriteOrderMode.PRIMARY;
-
                     // Use old value?
                     if (conflictCtx.isUseOld()) {
                         GridCacheVersion newConflictVer = conflictVer != null ? conflictVer : newVer;
@@ -2180,7 +2178,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                         if (!isNew() &&                                                                       // Not initial value,
                             verCheck &&                                                                       // and atomic version check,
                             oldConflictVer.dataCenterId() == newConflictVer.dataCenterId() &&                 // and data centers are equal,
-                            ATOMIC_VER_COMPARATOR.compare(oldConflictVer, newConflictVer, ignoreTime) == 0 && // and both versions are equal,
+                            ATOMIC_VER_COMPARATOR.compare(oldConflictVer, newConflictVer, true) == 0 && // and both versions are equal,
                             cctx.writeThrough() &&                                                            // and store is enabled,
                             primary)                                                                          // and we are primary.
                         {
@@ -2226,13 +2224,11 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                     conflictVer = null;
             }
 
-            boolean ignoreTime = cctx.config().getAtomicWriteOrderMode() == CacheAtomicWriteOrderMode.PRIMARY;
-
             // Perform version check only in case there was no explicit conflict resolution.
             if (conflictCtx == null) {
                 if (verCheck) {
-                    if (!isNew() && ATOMIC_VER_COMPARATOR.compare(ver, newVer, ignoreTime) >= 0) {
-                        if (ATOMIC_VER_COMPARATOR.compare(ver, newVer, ignoreTime) == 0 && cctx.writeThrough() && primary) {
+                    if (!isNew() && ATOMIC_VER_COMPARATOR.compare(ver, newVer, true) >= 0) {
+                        if (ATOMIC_VER_COMPARATOR.compare(ver, newVer, true) == 0 && cctx.writeThrough() && primary) {
                             if (log.isDebugEnabled())
                                 log.debug("Received entry update with same version as current (will update store) " +
                                     "[entry=" + this + ", newVer=" + newVer + ']');
@@ -2307,7 +2303,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                     }
                 }
                 else
-                    assert isNew() || ATOMIC_VER_COMPARATOR.compare(ver, newVer, ignoreTime) <= 0 :
+                    assert isNew() || ATOMIC_VER_COMPARATOR.compare(ver, newVer, true) <= 0 :
                         "Invalid version for inner update [isNew=" + isNew() + ", entry=" + this + ", newVer=" + newVer + ']';
             }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
index 0646d5a..4de465c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
@@ -50,7 +50,7 @@ public abstract class GridCacheMessage implements Message {
     private static final long serialVersionUID = 0L;
 
     /** Maximum number of cache lookup indexes. */
-    public static final int MAX_CACHE_MSG_LOOKUP_INDEX = 5;
+    public static final int MAX_CACHE_MSG_LOOKUP_INDEX = 7;
 
     /** Cache message index field name. */
     public static final String CACHE_MSG_INDEX_FIELD_NAME = "CACHE_MSG_IDX";
@@ -501,7 +501,7 @@ public abstract class GridCacheMessage implements Message {
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("ForLoopReplaceableByForEach")
-    protected final void prepareMarshalCacheObjects(@Nullable List<? extends CacheObject> col,
+    public final void prepareMarshalCacheObjects(@Nullable List<? extends CacheObject> col,
         GridCacheContext ctx) throws IgniteCheckedException {
         if (col == null)
             return;
@@ -553,7 +553,7 @@ public abstract class GridCacheMessage implements Message {
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("ForLoopReplaceableByForEach")
-    protected final void finishUnmarshalCacheObjects(@Nullable List<? extends CacheObject> col,
+    public final void finishUnmarshalCacheObjects(@Nullable List<? extends CacheObject> col,
         GridCacheContext ctx,
         ClassLoader ldr)
         throws IgniteCheckedException
@@ -701,6 +701,17 @@ public abstract class GridCacheMessage implements Message {
         return reader.afterMessageRead(GridCacheMessage.class);
     }
 
+    /**
+     * @param str Bulder.
+     * @param name Flag name.
+     */
+    protected final void appendFlag(StringBuilder str, String name) {
+        if (str.length() > 0)
+            str.append('|');
+
+        str.append(name);
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridCacheMessage.class, this, "cacheId", cacheId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
index 4ec13fc..dff2c88 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
@@ -29,6 +29,7 @@ import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.events.DiscoveryEvent;
@@ -105,9 +106,11 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     @GridToStringExclude
     private final ConcurrentMap<GridCacheVersion, Collection<GridCacheMvccFuture<?>>> mvccFuts = newMap();
 
+    /** */
+    private final AtomicLong atomicFutId = new AtomicLong(U.currentTimeMillis());
+
     /** Pending atomic futures. */
-    private final ConcurrentMap<GridCacheVersion, GridCacheAtomicFuture<?>> atomicFuts =
-        new ConcurrentHashMap8<>();
+    private final ConcurrentHashMap8<Long, GridCacheAtomicFuture<?>> atomicFuts = new ConcurrentHashMap8<>();
 
     /** Pending data streamer futures. */
     private final GridConcurrentHashSet<DataStreamerFuture> dataStreamerFuts = new GridConcurrentHashSet<>();
@@ -253,10 +256,10 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
                 cacheFut.onNodeLeft(discoEvt.eventNode().id());
 
                 if (cacheFut.isCancelled() || cacheFut.isDone()) {
-                    GridCacheVersion futVer = cacheFut.version();
+                    Long futId = cacheFut.id();
 
-                    if (futVer != null)
-                        atomicFuts.remove(futVer, cacheFut);
+                    if (futId != null)
+                        atomicFuts.remove(futId, cacheFut);
                 }
             }
         }
@@ -423,14 +426,21 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
-     * @param futVer Future ID.
+     * @return ID for atomic cache update future.
+     */
+    public long atomicFutureId() {
+        return atomicFutId.incrementAndGet();
+    }
+
+    /**
+     * @param futId Future ID.
      * @param fut Future.
      * @return {@code False} if future was forcibly completed with error.
      */
-    public boolean addAtomicFuture(GridCacheVersion futVer, GridCacheAtomicFuture<?> fut) {
-        IgniteInternalFuture<?> old = atomicFuts.put(futVer, fut);
+    public boolean addAtomicFuture(Long futId, GridCacheAtomicFuture<?> fut) {
+        IgniteInternalFuture<?> old = atomicFuts.put(futId, fut);
 
-        assert old == null : "Old future is not null [futVer=" + futVer + ", fut=" + fut + ", old=" + old + ']';
+        assert old == null : "Old future is not null [futId=" + futId + ", fut=" + fut + ", old=" + old + ']';
 
         return onFutureAdded(fut);
     }
@@ -443,6 +453,13 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
+     * @return Number of pending atomic futures.
+     */
+    public int atomicFuturesCount() {
+        return atomicFuts.size();
+    }
+
+    /**
      * @return Collection of pending data streamer futures.
      */
     public Collection<DataStreamerFuture> dataStreamerFutures() {
@@ -452,19 +469,19 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     /**
      * Gets future by given future ID.
      *
-     * @param futVer Future ID.
+     * @param futId Future ID.
      * @return Future.
      */
-    @Nullable public IgniteInternalFuture<?> atomicFuture(GridCacheVersion futVer) {
-        return atomicFuts.get(futVer);
+    @Nullable public IgniteInternalFuture<?> atomicFuture(Long futId) {
+        return atomicFuts.get(futId);
     }
 
     /**
-     * @param futVer Future ID.
+     * @param futId Future ID.
      * @return Removed future.
      */
-    @Nullable public IgniteInternalFuture<?> removeAtomicFuture(GridCacheVersion futVer) {
-        return atomicFuts.remove(futVer);
+    @Nullable public IgniteInternalFuture<?> removeAtomicFuture(Long futId) {
+        return atomicFuts.remove(futId);
     }
 
     /**
@@ -481,6 +498,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
 
     /**
      * @param topVer Topology version.
+     * @return Future.
      */
     public GridFutureAdapter addDataStreamerFuture(AffinityTopologyVersion topVer) {
         final DataStreamerFuture fut = new DataStreamerFuture(topVer);

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index f7ac812..c7ac31a 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -859,7 +859,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         if (!ctx.clientNode() && !ctx.isDaemon())
             addRemovedItemsCleanupTask(Long.getLong(IGNITE_CACHE_REMOVED_ENTRIES_TTL, 10_000));
-
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturn.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturn.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturn.java
index 02c882c..c5d4066 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturn.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturn.java
@@ -126,12 +126,10 @@ public class GridCacheReturn implements Externalizable, Message {
     }
 
     /**
-     * Checks if value is not {@code null}.
      *
-     * @return {@code True} if value is not {@code null}.
      */
-    public boolean hasValue() {
-        return v != null;
+    public boolean emptyResult() {
+        return !invokeRes && v  == null && cacheObj == null && success;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDeferredAckMessageSender.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDeferredAckMessageSender.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDeferredAckMessageSender.java
index 7145dc2..5265ec9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDeferredAckMessageSender.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDeferredAckMessageSender.java
@@ -41,30 +41,31 @@ public abstract class GridDeferredAckMessageSender {
     private GridTimeoutProcessor time;
 
     /** Closure processor. */
-    public GridClosureProcessor closure;
+    public GridClosureProcessor c;
 
     /**
      * @param time Time.
-     * @param closure Closure.
+     * @param c Closure.
      */
     public GridDeferredAckMessageSender(GridTimeoutProcessor time,
-        GridClosureProcessor closure) {
+        GridClosureProcessor c) {
         this.time = time;
-        this.closure = closure;
+        this.c = c;
     }
 
     /**
-     *
+     * @return Timeout.
      */
     public abstract int getTimeout();
 
     /**
-     *
+     * @return Buffer size.
      */
     public abstract int getBufferSize();
 
     /**
-     *
+     * @param nodeId Node ID.
+     * @param vers Versions to send.
      */
     public abstract void finish(UUID nodeId, ConcurrentLinkedDeque8<GridCacheVersion> vers);
 
@@ -151,7 +152,7 @@ public abstract class GridDeferredAckMessageSender {
         /** {@inheritDoc} */
         @Override public void onTimeout() {
             if (guard.compareAndSet(false, true)) {
-                closure.runLocalSafe(new Runnable() {
+                c.runLocalSafe(new Runnable() {
                     @Override public void run() {
                         writeLock().lock();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObjectImpl.java
index 146e554..4f8570c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObjectImpl.java
@@ -17,8 +17,10 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -29,7 +31,6 @@ public class KeyCacheObjectImpl extends CacheObjectAdapter implements KeyCacheOb
     private static final long serialVersionUID = 0L;
 
     /** */
-    @GridDirectTransient
     private int part = -1;
 
     /**
@@ -42,14 +43,6 @@ public class KeyCacheObjectImpl extends CacheObjectAdapter implements KeyCacheOb
     /**
      * @param val Value.
      * @param valBytes Value bytes.
-     */
-    public KeyCacheObjectImpl(Object val, byte[] valBytes) {
-        this(val, valBytes, -1);
-    }
-
-    /**
-     * @param val Value.
-     * @param valBytes Value bytes.
      * @param part Partition.
      */
     public KeyCacheObjectImpl(Object val, byte[] valBytes, int part) {
@@ -130,7 +123,57 @@ public class KeyCacheObjectImpl extends CacheObjectAdapter implements KeyCacheOb
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 1;
+        return 2;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 1:
+                part = reader.readInt("part");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(KeyCacheObjectImpl.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 1:
+                if (!writer.writeInt("part", part))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
index 2af822a..6ca15de 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
@@ -32,6 +32,7 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.managers.discovery.DiscoCache;
+import org.apache.ignite.internal.processors.affinity.AffinityAssignment;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId;
@@ -409,6 +410,13 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
+    @Nullable @Override public List<ClusterNode> nodes(int p,
+        AffinityAssignment affAssignment,
+        List<ClusterNode> affNodes) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
     @Override public List<ClusterNode> nodes(int p, AffinityTopologyVersion topVer) {
         lock.readLock().lock();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
index bdd84b0..605150a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
@@ -23,6 +23,7 @@ import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.affinity.AffinityAssignment;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap;
@@ -170,6 +171,14 @@ public interface GridDhtPartitionTopology {
 
     /**
      * @param p Partition ID.
+     * @param affAssignment Assignments.
+     * @param affNodes Node assigned for given partition by affinity.
+     * @return Collection of all nodes responsible for this partition with primary node being first.
+     */
+    @Nullable public List<ClusterNode> nodes(int p, AffinityAssignment affAssignment, List<ClusterNode> affNodes);
+
+    /**
+     * @param p Partition ID.
      * @return Collection of all nodes who {@code own} this partition.
      */
     public List<ClusterNode> owners(int p);

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index 49de280..53257d3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -828,11 +828,32 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
+    @Nullable @Override public List<ClusterNode> nodes(int p,
+        AffinityAssignment affAssignment,
+        List<ClusterNode> affNodes) {
+        return nodes0(p, affAssignment, affNodes);
+    }
+
+    /** {@inheritDoc} */
     @Override public List<ClusterNode> nodes(int p, AffinityTopologyVersion topVer) {
         AffinityAssignment affAssignment = cctx.affinity().assignment(topVer);
 
         List<ClusterNode> affNodes = affAssignment.get(p);
 
+        List<ClusterNode> nodes = nodes0(p, affAssignment, affNodes);
+
+        return nodes != null ? nodes : affNodes;
+    }
+
+    /**
+     * @param p Partition.
+     * @param affAssignment Assignments.
+     * @param affNodes Node assigned for given partition by affinity.
+     * @return Nodes responsible for given partition (primary is first).
+     */
+    @Nullable private List<ClusterNode> nodes0(int p, AffinityAssignment affAssignment, List<ClusterNode> affNodes) {
+        AffinityTopologyVersion topVer = affAssignment.topologyVersion();
+
         lock.readLock().lock();
 
         try {
@@ -866,7 +887,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                 }
             }
 
-            return nodes != null ? nodes : affNodes;
+            return nodes;
         }
         finally {
             lock.readLock().unlock();

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbc472fe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateFuture.java
index 4cb113e..5ff5aa4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateFuture.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
@@ -30,10 +31,13 @@ import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.affinity.AffinityAssignment;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheAtomicFuture;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
+import org.apache.ignite.internal.processors.cache.GridCacheReturn;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -41,14 +45,15 @@ import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.CI1;
-import org.apache.ignite.internal.util.typedef.CI2;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
 
 /**
  * DHT atomic cache backup update future.
@@ -74,56 +79,38 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte
     protected final GridCacheContext cctx;
 
     /** Future version. */
-    protected final GridCacheVersion futVer;
-
-    /** Completion callback. */
-    @GridToStringExclude
-    private final CI2<GridNearAtomicAbstractUpdateRequest, GridNearAtomicUpdateResponse> completionCb;
+    @GridToStringInclude
+    protected final long futId;
 
     /** Update request. */
-    protected final GridNearAtomicAbstractUpdateRequest updateReq;
-
-    /** Update response. */
-    final GridNearAtomicUpdateResponse updateRes;
+    final GridNearAtomicAbstractUpdateRequest updateReq;
 
     /** Mappings. */
-    @GridToStringInclude
+    @GridToStringExclude
     protected Map<UUID, GridDhtAtomicAbstractUpdateRequest> mappings;
 
     /** Continuous query closures. */
     private Collection<CI1<Boolean>> cntQryClsrs;
 
-    /** */
-    private final boolean waitForExchange;
-
     /** Response count. */
     private volatile int resCnt;
 
     /**
      * @param cctx Cache context.
-     * @param completionCb Callback to invoke when future is completed.
      * @param writeVer Write version.
      * @param updateReq Update request.
-     * @param updateRes Update response.
      */
     protected GridDhtAtomicAbstractUpdateFuture(
         GridCacheContext cctx,
-        CI2<GridNearAtomicAbstractUpdateRequest, GridNearAtomicUpdateResponse> completionCb,
         GridCacheVersion writeVer,
-        GridNearAtomicAbstractUpdateRequest updateReq,
-        GridNearAtomicUpdateResponse updateRes
+        GridNearAtomicAbstractUpdateRequest updateReq
     ) {
         this.cctx = cctx;
 
-        this.futVer = cctx.isLocalNode(updateRes.nodeId()) ?
-            cctx.versions().next(updateReq.topologyVersion()) : // Generate new if request mapped to local.
-            updateReq.futureVersion();
         this.updateReq = updateReq;
-        this.completionCb = completionCb;
-        this.updateRes = updateRes;
         this.writeVer = writeVer;
 
-        waitForExchange = !(updateReq.topologyLocked() || (updateReq.fastMap() && !updateReq.clientRequest()));
+        futId = cctx.mvcc().atomicFutureId();
 
         if (log == null) {
             msgLog = cctx.shared().atomicMessageLogger();
@@ -131,8 +118,15 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte
         }
     }
 
+    /**
+     * @return {@code True} if all updates are sent to DHT.
+     */
+    protected abstract boolean sendAllToDht();
+
     /** {@inheritDoc} */
     @Override public final IgniteInternalFuture<Void> completeFuture(AffinityTopologyVersion topVer) {
+        boolean waitForExchange = !updateReq.topologyLocked();
+
         if (waitForExchange && updateReq.topologyVersion().compareTo(topVer) < 0)
             return this;
 
@@ -141,17 +135,23 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte
 
     /**
      * @param clsr Continuous query closure.
+     * @param sync Synchronous continuous query flag.
      */
-    public final void addContinuousQueryClosure(CI1<Boolean> clsr) {
+    public final void addContinuousQueryClosure(CI1<Boolean> clsr, boolean sync) {
         assert !isDone() : this;
 
-        if (cntQryClsrs == null)
-            cntQryClsrs = new ArrayList<>(10);
+        if (sync)
+            clsr.apply(true);
+        else {
+            if (cntQryClsrs == null)
+                cntQryClsrs = new ArrayList<>(10);
 
-        cntQryClsrs.add(clsr);
+            cntQryClsrs.add(clsr);
+        }
     }
 
     /**
+     * @param affAssignment Affinity assignment.
      * @param entry Entry to map.
      * @param val Value to write.
      * @param entryProcessor Entry processor.
@@ -163,7 +163,9 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte
      * @param updateCntr Partition update counter.
      */
     @SuppressWarnings("ForLoopReplaceableByForEach")
-    final void addWriteEntry(GridDhtCacheEntry entry,
+    final void addWriteEntry(
+        AffinityAssignment affAssignment,
+        GridDhtCacheEntry entry,
         @Nullable CacheObject val,
         EntryProcessor<Object, Object, Object> entryProcessor,
         long ttl,
@@ -174,7 +176,12 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte
         long updateCntr) {
         AffinityTopologyVersion topVer = updateReq.topologyVersion();
 
-        List<ClusterNode> dhtNodes = cctx.dht().topology().nodes(entry.partition(), topVer);
+        List<ClusterNode> affNodes = affAssignment.get(entry.partition());
+
+        List<ClusterNode> dhtNodes = cctx.dht().topology().nodes(entry.partition(), affAssignment, affNodes);
+
+        if (dhtNodes == null)
+            dhtNodes = affNodes;
 
         if (log.isDebugEnabled())
             log.debug("Mapping entry to DHT nodes [nodes=" + U.nodeIds(dhtNodes) + ", entry=" + entry + ']');
@@ -193,8 +200,8 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte
 
                 if (updateReq == null) {
                     updateReq = createRequest(
-                        node,
-                        futVer,
+                        node.id(),
+                        futId,
                         writeVer,
                         syncMode,
                         topVer,
@@ -212,7 +219,6 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte
                     conflictExpireTime,
                     conflictVer,
                     addPrevVal,
-                    entry.partition(),
                     prevVal,
                     updateCntr);
             }
@@ -239,7 +245,8 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte
      * @param ttl TTL for near cache update (optional).
      * @param expireTime Expire time for near cache update (optional).
      */
-    final void addNearWriteEntries(Collection<UUID> readers,
+    final void addNearWriteEntries(
+        Collection<UUID> readers,
         GridDhtCacheEntry entry,
         @Nullable CacheObject val,
         EntryProcessor<Object, Object, Object> entryProcessor,
@@ -262,8 +269,8 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte
                     continue;
 
                 updateReq = createRequest(
-                    node,
-                    futVer,
+                    node.id(),
+                    futId,
                     writeVer,
                     syncMode,
                     topVer,
@@ -274,8 +281,6 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte
                 mappings.put(nodeId, updateReq);
             }
 
-            addNearReaderEntry(entry);
-
             updateReq.addNearWriteValue(entry.key(),
                 val,
                 entryProcessor,
@@ -284,12 +289,15 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte
         }
     }
 
-    /**
-     * adds new nearReader.
-     *
-     * @param entry GridDhtCacheEntry.
-     */
-    protected abstract void addNearReaderEntry(GridDhtCacheEntry entry);
+    /** {@inheritDoc} */
+    @Override public final IgniteUuid futureId() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public final Long id() {
+        return futId;
+    }
 
     /**
      * @return Write version.
@@ -299,21 +307,11 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte
     }
 
     /** {@inheritDoc} */
-    @Override public final IgniteUuid futureId() {
-        return futVer.asGridUuid();
-    }
-
-    /** {@inheritDoc} */
-    @Override public final GridCacheVersion version() {
-        return futVer;
-    }
-
-    /** {@inheritDoc} */
     @Override public final boolean onNodeLeft(UUID nodeId) {
         boolean res = registerResponse(nodeId);
 
         if (res && msgLog.isDebugEnabled()) {
-            msgLog.debug("DTH update fut, node left [futId=" + futVer + ", writeVer=" + writeVer +
+            msgLog.debug("DTH update fut, node left [futId=" + futId + ", writeVer=" + writeVer +
                 ", node=" + nodeId + ']');
         }
 
@@ -324,7 +322,7 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte
      * @param nodeId Node ID.
      * @return {@code True} if request found.
      */
-    final boolean registerResponse(UUID nodeId) {
+    private boolean registerResponse(UUID nodeId) {
         int resCnt0;
 
         GridDhtAtomicAbstractUpdateRequest req = mappings != null ? mappings.get(nodeId) : null;
@@ -353,41 +351,103 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte
 
     /**
      * Sends requests to remote nodes.
+     *
+     * @param nearNode Near node.
+     * @param ret Cache operation return value.
+     * @param updateRes Response.
+     * @param completionCb Callback to invoke to send response to near node.
+     */
+    final void map(ClusterNode nearNode,
+        GridCacheReturn ret,
+        GridNearAtomicUpdateResponse updateRes,
+        GridDhtAtomicCache.UpdateReplyClosure completionCb) {
+        if (F.isEmpty(mappings)) {
+            updateRes.dhtNodes(Collections.<UUID>emptyList());
+
+            completionCb.apply(updateReq, updateRes);
+
+            onDone();
+
+            return;
+        }
+
+        boolean needReplyToNear = updateReq.writeSynchronizationMode() == PRIMARY_SYNC ||
+            !ret.emptyResult() ||
+            updateRes.nearVersion() != null ||
+            cctx.localNodeId().equals(nearNode.id());
+
+        boolean needMapping = updateReq.fullSync() && (updateReq.needPrimaryResponse() || !sendAllToDht());
+
+        if (needMapping) {
+            initMapping(updateRes);
+
+            needReplyToNear = true;
+        }
+
+        sendDhtRequests(nearNode, ret);
+
+        if (needReplyToNear)
+            completionCb.apply(updateReq, updateRes);
+    }
+
+    /**
+     * @param updateRes Response.
      */
-    final void map() {
+    private void initMapping(GridNearAtomicUpdateResponse updateRes) {
+        List<UUID> dhtNodes;
+
         if (!F.isEmpty(mappings)) {
-            for (GridDhtAtomicAbstractUpdateRequest req : mappings.values()) {
-                try {
-                    cctx.io().send(req.nodeId(), req, cctx.ioPolicy());
+            dhtNodes = new ArrayList<>(mappings.size());
 
-                    if (msgLog.isDebugEnabled()) {
-                        msgLog.debug("DTH update fut, sent request [futId=" + futVer +
-                            ", writeVer=" + writeVer + ", node=" + req.nodeId() + ']');
-                    }
+            dhtNodes.addAll(mappings.keySet());
+        }
+        else
+            dhtNodes = Collections.emptyList();
+
+        updateRes.dhtNodes(dhtNodes);
+    }
+
+    /**
+     * @param nearNode Near node.
+     * @param ret Return value.
+     */
+    private void sendDhtRequests(ClusterNode nearNode, GridCacheReturn ret) {
+        for (GridDhtAtomicAbstractUpdateRequest req : mappings.values()) {
+            try {
+                assert !cctx.localNodeId().equals(req.nodeId()) : req;
+
+                if (updateReq.fullSync()) {
+                    req.nearReplyInfo(nearNode.id(), updateReq.futureId());
+
+                    if (ret.emptyResult())
+                        req.hasResult(true);
                 }
-                catch (ClusterTopologyCheckedException ignored) {
-                    if (msgLog.isDebugEnabled()) {
-                        msgLog.debug("DTH update fut, failed to send request, node left [futId=" + futVer +
-                            ", writeVer=" + writeVer + ", node=" + req.nodeId() + ']');
-                    }
 
-                    registerResponse(req.nodeId());
+                if (cntQryClsrs != null)
+                    req.replyWithoutDelay(true);
+
+                cctx.io().send(req.nodeId(), req, cctx.ioPolicy());
+
+                if (msgLog.isDebugEnabled()) {
+                    msgLog.debug("DTH update fut, sent request [futId=" + futId +
+                        ", writeVer=" + writeVer + ", node=" + req.nodeId() + ']');
                 }
-                catch (IgniteCheckedException ignored) {
-                    U.error(msgLog, "Failed to send request [futId=" + futVer +
+            }
+            catch (ClusterTopologyCheckedException ignored) {
+                if (msgLog.isDebugEnabled()) {
+                    msgLog.debug("DTH update fut, failed to send request, node left [futId=" + futId +
                         ", writeVer=" + writeVer + ", node=" + req.nodeId() + ']');
-
-                    registerResponse(req.nodeId());
                 }
+
+                registerResponse(req.nodeId());
             }
-        }
-        else
-            onDone();
+            catch (IgniteCheckedException ignored) {
+                U.error(msgLog, "Failed to send request [futId=" + futId +
+                    ", writeVer=" + writeVer + ", node=" + req.nodeId() + ']');
 
-        // Send response right away if no ACKs from backup is required.
-        // Backups will send ACKs anyway, future will be completed after all backups have replied.
-        if (updateReq.writeSynchronizationMode() != FULL_SYNC)
-            completionCb.apply(updateReq, updateRes);
+                registerResponse(req.nodeId());
+            }
+        }
     }
 
     /**
@@ -395,7 +455,7 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte
      *
      * @param nodeId Backup node ID.
      */
-    public final void onResult(UUID nodeId) {
+    final void onDeferredResponse(UUID nodeId) {
         if (log.isDebugEnabled())
             log.debug("Received deferred DHT atomic update future result [nodeId=" + nodeId + ']');
 
@@ -403,8 +463,31 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte
     }
 
     /**
-     * @param node Node.
-     * @param futVer Future version.
+     * @param nodeId Node ID.
+     * @param res Response.
+     */
+    final void onDhtResponse(UUID nodeId, GridDhtAtomicUpdateResponse res) {
+        if (!F.isEmpty(res.nearEvicted())) {
+            for (KeyCacheObject key : res.nearEvicted()) {
+                try {
+                    GridDhtCacheEntry entry = (GridDhtCacheEntry)cctx.cache().peekEx(key);
+
+                    if (entry != null)
+                        entry.removeReader(nodeId, res.messageId());
+                }
+                catch (GridCacheEntryRemovedException e) {
+                    if (log.isDebugEnabled())
+                        log.debug("Entry with evicted reader was removed [key=" + key + ", err=" + e + ']');
+                }
+            }
+        }
+
+        registerResponse(nodeId);
+    }
+
+    /**
+     * @param nodeId Node ID.
+     * @param futId Future ID.
      * @param writeVer Update version.
      * @param syncMode Write synchronization mode.
      * @param topVer Topology version.
@@ -414,8 +497,8 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte
      * @return Request.
      */
     protected abstract GridDhtAtomicAbstractUpdateRequest createRequest(
-        ClusterNode node,
-        GridCacheVersion futVer,
+        UUID nodeId,
+        long futId,
         GridCacheVersion writeVer,
         CacheWriteSynchronizationMode syncMode,
         @NotNull AffinityTopologyVersion topVer,
@@ -424,38 +507,18 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte
         @Nullable GridCacheVersion conflictVer
     );
 
-    /**
-     * Callback for backup update response.
-     *
-     * @param nodeId Backup node ID.
-     * @param updateRes Update response.
-     */
-    public abstract void onResult(UUID nodeId, GridDhtAtomicUpdateResponse updateRes);
-
-    /**
-     * @param updateRes Response.
-     * @param err Error.
-     */
-    protected abstract void addFailedKeys(GridNearAtomicUpdateResponse updateRes, Throwable err);
-
     /** {@inheritDoc} */
     @Override public final boolean onDone(@Nullable Void res, @Nullable Throwable err) {
         if (super.onDone(res, err)) {
-            cctx.mvcc().removeAtomicFuture(version());
+            cctx.mvcc().removeAtomicFuture(futId);
 
             boolean suc = err == null;
 
-            if (!suc)
-                addFailedKeys(updateRes, err);
-
             if (cntQryClsrs != null) {
                 for (CI1<Boolean> clsr : cntQryClsrs)
                     clsr.apply(suc);
             }
 
-            if (updateReq.writeSynchronizationMode() == FULL_SYNC)
-                completionCb.apply(updateReq, updateRes);
-
             return true;
         }
 
@@ -471,4 +534,21 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte
     @Override public void markNotTrackable() {
         // No-op.
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        synchronized (this) {
+            Map<UUID, String> dhtRes = F.viewReadOnly(mappings,
+                new IgniteClosure<GridDhtAtomicAbstractUpdateRequest, String>() {
+                    @Override public String apply(GridDhtAtomicAbstractUpdateRequest req) {
+                        return "[res=" + req.hasResponse() +
+                            ", size=" + req.size() +
+                            ", nearSize=" + req.nearSize() + ']';
+                    }
+                }
+            );
+
+            return S.toString(GridDhtAtomicAbstractUpdateFuture.class, this, "dhtRes", dhtRes);
+        }
+    }
 }