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 2016/02/18 07:04:20 UTC

[01/21] ignite git commit: Fixed IGNITE-2604 "CacheContinuousQueryBatchAck is sent to nodes that doesn't hold cache data".

Repository: ignite
Updated Branches:
  refs/heads/ignite-1232 86bddb94a -> fce714c06


Fixed IGNITE-2604 "CacheContinuousQueryBatchAck is sent to nodes that doesn't hold cache data".


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

Branch: refs/heads/ignite-1232
Commit: 675a7c1fc321d7f3d19202a5055df3f6076e1fd6
Parents: 6247ac7
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Tue Feb 16 13:47:44 2016 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Tue Feb 16 13:47:44 2016 +0300

----------------------------------------------------------------------
 .../continuous/CacheContinuousQueryHandler.java |   4 +-
 .../continuous/CacheContinuousQueryManager.java |  21 +-
 .../continuous/CacheContinuousBatchAckTest.java | 355 +++++++++++++++++++
 ...heContinuousBatchForceServerModeAckTest.java |  80 +++++
 .../CacheContinuousQueryLostPartitionTest.java  |   4 +-
 .../IgniteCacheQuerySelfTestSuite.java          |   4 +
 6 files changed, 456 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/675a7c1f/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 cf9b439..498f37d 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
@@ -1034,10 +1034,10 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
                     Collection<ClusterNode> nodes = new HashSet<>();
 
                     for (AffinityTopologyVersion topVer : t.get2())
-                        nodes.addAll(ctx.discovery().cacheNodes(topVer));
+                        nodes.addAll(ctx.discovery().cacheAffinityNodes(cctx.name(), topVer));
 
                     for (ClusterNode node : nodes) {
-                        if (!node.id().equals(ctx.localNodeId())) {
+                        if (!node.isLocal()) {
                             try {
                                 cctx.io().send(node, msg, GridIoPolicy.SYSTEM_POOL);
                             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/675a7c1f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
index cc59989..968fc23 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
@@ -56,6 +56,7 @@ import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.continuous.GridContinuousHandler;
 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.U;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.plugin.security.SecurityPermission;
@@ -116,17 +117,19 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
         // Append cache name to the topic.
         topicPrefix = "CONTINUOUS_QUERY" + (cctx.name() == null ? "" : "_" + cctx.name());
 
-        cctx.io().addHandler(cctx.cacheId(), CacheContinuousQueryBatchAck.class,
-            new CI2<UUID, CacheContinuousQueryBatchAck>() {
-                @Override public void apply(UUID uuid, CacheContinuousQueryBatchAck msg) {
-                    CacheContinuousQueryListener lsnr = lsnrs.get(msg.routineId());
+        if (cctx.affinityNode()) {
+            cctx.io().addHandler(cctx.cacheId(), CacheContinuousQueryBatchAck.class,
+                new CI2<UUID, CacheContinuousQueryBatchAck>() {
+                    @Override public void apply(UUID uuid, CacheContinuousQueryBatchAck msg) {
+                        CacheContinuousQueryListener lsnr = lsnrs.get(msg.routineId());
 
-                    if (lsnr != null)
-                        lsnr.cleanupBackupQueue(msg.updateCntrs());
-                }
-            });
+                        if (lsnr != null)
+                            lsnr.cleanupBackupQueue(msg.updateCntrs());
+                    }
+                });
 
-        cctx.time().schedule(new BackupCleaner(lsnrs, cctx.kernalContext()), BACKUP_ACK_FREQ, BACKUP_ACK_FREQ);
+            cctx.time().schedule(new BackupCleaner(lsnrs, cctx.kernalContext()), BACKUP_ACK_FREQ, BACKUP_ACK_FREQ);
+        }
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/675a7c1f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousBatchAckTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousBatchAckTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousBatchAckTest.java
new file mode 100644
index 0000000..c69ccf2
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousBatchAckTest.java
@@ -0,0 +1,355 @@
+/*
+ * 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.query.continuous;
+
+import java.io.Serializable;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicBoolean;
+import javax.cache.event.CacheEntryListenerException;
+import javax.cache.event.CacheEntryUpdatedListener;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.ContinuousQuery;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.managers.communication.GridIoMessage;
+import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.util.typedef.PA;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+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.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
+import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_GRID_NAME;
+
+/**
+ * Continuous queries tests.
+ */
+public class CacheContinuousBatchAckTest extends GridCommonAbstractTest implements Serializable {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    protected static final String CLIENT = "_client";
+
+    /** */
+    protected static final String SERVER = "server";
+
+    /** */
+    protected static final String SERVER2 = "server2";
+
+    /** */
+    protected static final AtomicBoolean fail = new AtomicBoolean(false);
+
+    /** */
+    protected static final AtomicBoolean filterOn = new AtomicBoolean(false);
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (gridName.endsWith(CLIENT)) {
+            cfg.setClientMode(true);
+
+            cfg.setCommunicationSpi(new FailedTcpCommunicationSpi(true, false));
+        }
+        else if (gridName.endsWith(SERVER2))
+            cfg.setCommunicationSpi(new FailedTcpCommunicationSpi(false, true));
+        else
+            cfg.setCommunicationSpi(new FailedTcpCommunicationSpi(false, false));
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrid(SERVER);
+        startGrid(SERVER2);
+        startGrid("1" + CLIENT);
+        startGrid("2" + CLIENT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        fail.set(false);
+
+        filterOn.set(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartition() throws Exception {
+        checkBackupAcknowledgeMessage(cacheConfiguration(PARTITIONED, 1, ATOMIC, ONHEAP_TIERED, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionWithFilter() throws Exception {
+        filterOn.set(true);
+
+        checkBackupAcknowledgeMessage(cacheConfiguration(PARTITIONED, 1, ATOMIC, ONHEAP_TIERED, true));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionNoBackups() throws Exception {
+        checkBackupAcknowledgeMessage(cacheConfiguration(PARTITIONED, 0, ATOMIC, ONHEAP_TIERED, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionTx() throws Exception {
+        checkBackupAcknowledgeMessage(cacheConfiguration(PARTITIONED, 1, TRANSACTIONAL, ONHEAP_TIERED, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionTxWithFilter() throws Exception {
+        filterOn.set(true);
+
+        checkBackupAcknowledgeMessage(cacheConfiguration(PARTITIONED, 1, TRANSACTIONAL, ONHEAP_TIERED, true));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionTxNoBackup() throws Exception {
+        checkBackupAcknowledgeMessage(cacheConfiguration(PARTITIONED, 0, TRANSACTIONAL, ONHEAP_TIERED, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionTxNoBackupWithFilter() throws Exception {
+        filterOn.set(true);
+
+        checkBackupAcknowledgeMessage(cacheConfiguration(PARTITIONED, 0, TRANSACTIONAL, ONHEAP_TIERED, true));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionOffheap() throws Exception {
+        checkBackupAcknowledgeMessage(cacheConfiguration(PARTITIONED, 1, ATOMIC, OFFHEAP_TIERED, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionOffheapWithFilter() throws Exception {
+        filterOn.set(true);
+
+        checkBackupAcknowledgeMessage(cacheConfiguration(PARTITIONED, 1, ATOMIC, OFFHEAP_TIERED, true));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionTxOffheap() throws Exception {
+        checkBackupAcknowledgeMessage(cacheConfiguration(PARTITIONED, 1, TRANSACTIONAL, OFFHEAP_TIERED, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReplicated() throws Exception {
+        checkBackupAcknowledgeMessage(cacheConfiguration(REPLICATED, 1, ATOMIC, ONHEAP_TIERED, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReplicatedTx() throws Exception {
+        checkBackupAcknowledgeMessage(cacheConfiguration(REPLICATED, 1, TRANSACTIONAL, ONHEAP_TIERED, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReplicatedTxWithFilter() throws Exception {
+        filterOn.set(true);
+
+        checkBackupAcknowledgeMessage(cacheConfiguration(REPLICATED, 1, TRANSACTIONAL, ONHEAP_TIERED, true));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReplicatedOffheap() throws Exception {
+        checkBackupAcknowledgeMessage(cacheConfiguration(REPLICATED, 1, ATOMIC, OFFHEAP_TIERED, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReplicatedTxOffheap() throws Exception {
+        checkBackupAcknowledgeMessage(cacheConfiguration(REPLICATED, 1, TRANSACTIONAL, OFFHEAP_TIERED, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReplicatedTxOffheapWithFilter() throws Exception {
+        filterOn.set(true);
+
+        checkBackupAcknowledgeMessage(cacheConfiguration(REPLICATED, 1, TRANSACTIONAL, OFFHEAP_TIERED, true));
+    }
+
+    /**
+     * @param ccfg Cache configuration.
+     * @throws Exception If failed.
+     */
+    private void checkBackupAcknowledgeMessage(CacheConfiguration<Object, Object> ccfg) throws Exception {
+        QueryCursor qry = null;
+
+        IgniteCache<Object, Object> cache = null;
+
+        try {
+            ContinuousQuery q = new ContinuousQuery();
+
+            q.setLocalListener(new CacheEntryUpdatedListener() {
+                @Override public void onUpdated(Iterable iterable) throws CacheEntryListenerException {
+                    // No-op.
+                }
+            });
+
+            cache = grid(SERVER).getOrCreateCache(ccfg);
+
+            qry = cache.query(q);
+
+            for (int i = 0; i < 10000; i++)
+                cache.put(i, i);
+
+            assert !GridTestUtils.waitForCondition(new PA() {
+                @Override public boolean apply() {
+                    return fail.get();
+                }
+            }, 1300L);
+        }
+        finally {
+            if (qry != null)
+                qry.close();
+
+            if (cache != null)
+                grid(SERVER).destroyCache(cache.getName());
+        }
+    }
+
+    /**
+     *
+     * @param cacheMode Cache mode.
+     * @param backups Number of backups.
+     * @param atomicityMode Cache atomicity mode.
+     * @param memoryMode Cache memory mode.
+     * @param filter Filter enabled.
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<Object, Object> cacheConfiguration(
+        CacheMode cacheMode,
+        int backups,
+        CacheAtomicityMode atomicityMode,
+        CacheMemoryMode memoryMode, boolean filter) {
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
+
+        ccfg.setAtomicityMode(atomicityMode);
+        ccfg.setCacheMode(cacheMode);
+        ccfg.setMemoryMode(memoryMode);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+
+        if (cacheMode == PARTITIONED)
+            ccfg.setBackups(backups);
+
+        if (filter)
+            ccfg.setNodeFilter(new P1<ClusterNode>() {
+                @Override public boolean apply(ClusterNode node) {
+                    return !node.attributes().get(ATTR_GRID_NAME).equals(SERVER2);
+                }
+            });
+
+        return ccfg;
+    }
+
+    /**
+     *
+     */
+    protected static class FailedTcpCommunicationSpi extends TcpCommunicationSpi {
+        /** */
+        private boolean check;
+
+        /** */
+        private boolean periodicCheck;
+
+        /**
+         * @param alwaysCheck Always check inbound message.
+         * @param periodicCheck Check when {@code filterOn} enabled.
+         */
+        public FailedTcpCommunicationSpi(boolean alwaysCheck, boolean periodicCheck) {
+            this.check = alwaysCheck;
+            this.periodicCheck = periodicCheck;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void notifyListener(UUID sndId, Message msg, IgniteRunnable msgC) {
+            if (check || (periodicCheck && filterOn.get())) {
+                if (msg instanceof GridIoMessage &&
+                    ((GridIoMessage)msg).message() instanceof CacheContinuousQueryBatchAck)
+                    fail.set(true);
+            }
+
+            super.notifyListener(sndId, msg, msgC);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/675a7c1f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousBatchForceServerModeAckTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousBatchForceServerModeAckTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousBatchForceServerModeAckTest.java
new file mode 100644
index 0000000..f1794fa
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousBatchForceServerModeAckTest.java
@@ -0,0 +1,80 @@
+/*
+ * 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.query.continuous;
+
+import java.io.Serializable;
+import org.apache.ignite.configuration.IgniteConfiguration;
+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;
+
+/**
+ * Continuous queries tests.
+ */
+public class CacheContinuousBatchForceServerModeAckTest extends CacheContinuousBatchAckTest implements Serializable {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (gridName.endsWith(CLIENT)) {
+            cfg.setClientMode(true);
+
+            FailedTcpCommunicationSpi spi = new FailedTcpCommunicationSpi(true, false);
+
+            cfg.setCommunicationSpi(spi);
+
+            TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+            disco.setForceServerMode(true);
+
+            disco.setIpFinder(IP_FINDER);
+
+            cfg.setDiscoverySpi(disco);
+        }
+        else if (gridName.endsWith(SERVER2)) {
+            cfg.setCommunicationSpi(new FailedTcpCommunicationSpi(false, true));
+
+            TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+            disco.setIpFinder(IP_FINDER);
+
+            cfg.setDiscoverySpi(disco);
+        }
+        else {
+            cfg.setCommunicationSpi(new FailedTcpCommunicationSpi(false, false));
+
+            TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+            disco.setIpFinder(IP_FINDER);
+
+            cfg.setDiscoverySpi(disco);
+        }
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/675a7c1f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryLostPartitionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryLostPartitionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryLostPartitionTest.java
index 30613a4..f4659dc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryLostPartitionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryLostPartitionTest.java
@@ -138,7 +138,9 @@ public class CacheContinuousQueryLostPartitionTest extends GridCommonAbstractTes
         }, 2000L) : "Expected no create events, but got: " + lsnr2.createdCnt.get();
 
         // node2 now becomes the primary for the key.
-        grid(0).close();
+        stopGrid(0);
+
+        awaitPartitionMapExchange();
 
         cache2.put(key, "2");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/675a7c1f/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 84a9a45..c67a8cf 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -72,6 +72,8 @@ import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalAtomicQ
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.query.GridCacheSwapScanQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousBatchAckTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousBatchForceServerModeAckTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverAtomicPrimaryWriteOrderOffheapTieredTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverAtomicPrimaryWriteOrderSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverAtomicReplicatedSelfTest;
@@ -214,6 +216,8 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(CacheContinuousQueryFailoverAtomicPrimaryWriteOrderOffheapTieredTest.class);
         suite.addTestSuite(CacheContinuousQueryFailoverTxOffheapTieredTest.class);
         suite.addTestSuite(CacheContinuousQueryRandomOperationsTest.class);
+        suite.addTestSuite(CacheContinuousBatchAckTest.class);
+        suite.addTestSuite(CacheContinuousBatchForceServerModeAckTest.class);
 
         // Reduce fields queries.
         suite.addTestSuite(GridCacheReduceFieldsQueryLocalSelfTest.class);


[17/21] ignite git commit: Test investigation fix.

Posted by sb...@apache.org.
Test investigation fix.


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

Branch: refs/heads/ignite-1232
Commit: 2ad4b5cfc7d8f5d3fcf7f0233129078237129beb
Parents: d2e6535
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Feb 17 19:08:06 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Feb 17 19:08:06 2016 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/preloader/GridDhtPartitionDemander.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2ad4b5cf/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index 9634b0b..1354d96 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -975,7 +975,8 @@ public class GridDhtPartitionDemander {
                 if (parts != null) {
                     boolean rmvd = parts.remove(p);
 
-                    assert rmvd;
+                    assert rmvd : "Partition already done [cache=" + cctx.name() + ", fromNode=" + nodeId +
+                        ", part=" + p + ", left=" + parts + "]";
 
                     if (parts.isEmpty()) {
                         U.log(log, "Completed " + ((remaining.size() == 1 ? "(final) " : "") +


[04/21] ignite git commit: Fixed "IGNITE-2515 Make 'updateCntr' available through CacheContinuousQueryEvent public API"

Posted by sb...@apache.org.
Fixed "IGNITE-2515 Make 'updateCntr' available through CacheContinuousQueryEvent public API"


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

Branch: refs/heads/ignite-1232
Commit: cb35e1d7eaaea470e0afca99d5de9b0aec3e58ae
Parents: 60b6f09
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Tue Feb 16 17:44:38 2016 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Tue Feb 16 17:44:38 2016 +0300

----------------------------------------------------------------------
 .../cache/query/CacheQueryEntryEvent.java       |  48 +
 .../continuous/CacheContinuousQueryEvent.java   |  17 +-
 .../continuous/CacheContinuousQueryHandler.java |  95 +-
 .../CacheContinuousQueryListener.java           |   3 +-
 .../continuous/CacheContinuousQueryManager.java |  26 +-
 ...CacheContinuousQueryCounterAbstractTest.java | 613 +++++++++++++
 ...inuousQueryCounterPartitionedAtomicTest.java |  41 +
 ...ContinuousQueryCounterPartitionedTxTest.java |  41 +
 ...tinuousQueryCounterReplicatedAtomicTest.java |  41 +
 ...eContinuousQueryCounterReplicatedTxTest.java |  41 +
 ...acheContinuousQueryRandomOperationsTest.java | 896 ++++++++++++++++---
 .../IgniteCacheQuerySelfTestSuite.java          |   8 +
 12 files changed, 1652 insertions(+), 218 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cb35e1d7/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryEntryEvent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryEntryEvent.java b/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryEntryEvent.java
new file mode 100644
index 0000000..2c1c5e6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryEntryEvent.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.query;
+
+import javax.cache.Cache;
+import javax.cache.event.CacheEntryEvent;
+import javax.cache.event.EventType;
+
+/**
+ * A Cache continuous query entry event.
+ *
+ * @param <K> the type of key
+ * @param <V> the type of value
+ */
+public abstract class CacheQueryEntryEvent<K, V> extends CacheEntryEvent<K, V> {
+    /**
+     * Constructs a cache entry event from a given cache as source.
+     *
+     * @param source the cache that originated the event
+     * @param eventType Event type.
+     */
+    public CacheQueryEntryEvent(Cache source, EventType eventType) {
+        super(source, eventType);
+    }
+
+    /**
+     * Each cache update increases partition counter. The same cache updates have on the same value of counter
+     * on primary and backup nodes. This value can be useful to communicate with external applications.
+     *
+     * @return Value of counter for this event.
+     */
+    public abstract long getPartitionUpdateCounter();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cb35e1d7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEvent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEvent.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEvent.java
index d26e666..eab5dbd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEvent.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEvent.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.processors.cache.query.continuous;
 
 import javax.cache.Cache;
-import javax.cache.event.CacheEntryEvent;
+import org.apache.ignite.cache.query.CacheQueryEntryEvent;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -26,7 +26,7 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 /**
  * Continuous query event.
  */
-class CacheContinuousQueryEvent<K, V> extends CacheEntryEvent<K, V> {
+class CacheContinuousQueryEvent<K, V> extends CacheQueryEntryEvent<K, V> {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -57,8 +57,7 @@ class CacheContinuousQueryEvent<K, V> extends CacheEntryEvent<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override
-    public K getKey() {
+    @Override public K getKey() {
         return (K)cctx.cacheObjectContext().unwrapBinaryIfNeeded(e.key(), e.isKeepBinary(), false);
     }
 
@@ -68,8 +67,7 @@ class CacheContinuousQueryEvent<K, V> extends CacheEntryEvent<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override
-    public V getOldValue() {
+    @Override public V getOldValue() {
         return (V)cctx.cacheObjectContext().unwrapBinaryIfNeeded(e.oldValue(), e.isKeepBinary(), false);
     }
 
@@ -79,8 +77,13 @@ class CacheContinuousQueryEvent<K, V> extends CacheEntryEvent<K, V> {
     }
 
     /** {@inheritDoc} */
+    @Override public long getPartitionUpdateCounter() {
+        return e.updateCounter();
+    }
+
+    /** {@inheritDoc} */
     @Override public <T> T unwrap(Class<T> cls) {
-        if(cls.isAssignableFrom(getClass()))
+        if (cls.isAssignableFrom(getClass()))
             return cls.cast(this);
 
         throw new IllegalArgumentException("Unwrapping to class is not supported: " + cls);

http://git-wip-us.apache.org/repos/asf/ignite/blob/cb35e1d7/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 498f37d..08fe62a 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
@@ -325,9 +325,9 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
                 // skipPrimaryCheck is set only when listen locally for replicated cache events.
                 assert !skipPrimaryCheck || (cctx.isReplicated() && ctx.localNodeId().equals(nodeId));
 
-                boolean notify = true;
+                boolean notify = !evt.entry().isFiltered();
 
-                if (rmtFilter != null) {
+                if (notify && rmtFilter != null) {
                     try {
                         notify = rmtFilter.evaluate(evt);
                     }
@@ -472,38 +472,15 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
                 sendBackupAcknowledge(ackBuf.acknowledgeOnTimeout(), routineId, ctx);
             }
 
-            @Override public void skipUpdateEvent(CacheContinuousQueryEvent<K, V> evt, AffinityTopologyVersion topVer) {
-                try {
-                    assert evt != null;
-
-                    CacheContinuousQueryEntry e = evt.entry();
-
-                    EntryBuffer buf = entryBufs.get(e.partition());
+            @Override public void skipUpdateEvent(CacheContinuousQueryEvent<K, V> evt, AffinityTopologyVersion topVer,
+                boolean primary) {
+                assert evt != null;
 
-                    if (buf == null) {
-                        buf = new EntryBuffer();
-
-                        EntryBuffer oldRec = entryBufs.putIfAbsent(e.partition(), buf);
-
-                        if (oldRec != null)
-                            buf = oldRec;
-                    }
+                CacheContinuousQueryEntry e = evt.entry();
 
-                    e = buf.skipEntry(e);
-
-                    if (e != null && !ctx.localNodeId().equals(nodeId))
-                        ctx.continuous().addNotification(nodeId, routineId, e, topic, sync, true);
-                }
-                catch (ClusterTopologyCheckedException ex) {
-                    IgniteLogger log = ctx.log(getClass());
+                e.markFiltered();
 
-                    if (log.isDebugEnabled())
-                        log.debug("Failed to send event notification to node, node left cluster " +
-                                "[node=" + nodeId + ", err=" + ex + ']');
-                }
-                catch (IgniteCheckedException ex) {
-                    U.error(ctx.log(getClass()), "Failed to send event notification to node: " + nodeId, ex);
-                }
+                onEntryUpdated(evt, primary, false);
             }
 
             @Override public void onPartitionEvicted(int part) {
@@ -618,20 +595,22 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
         for (CacheContinuousQueryEntry e : entries)
             entries0.addAll(handleEvent(ctx, e));
 
-        Iterable<CacheEntryEvent<? extends K, ? extends V>> evts = F.viewReadOnly(entries0,
-            new C1<CacheContinuousQueryEntry, CacheEntryEvent<? extends K, ? extends V>>() {
-                @Override public CacheEntryEvent<? extends K, ? extends V> apply(CacheContinuousQueryEntry e) {
-                    return new CacheContinuousQueryEvent<>(cache, cctx, e);
-                }
-            },
-            new IgnitePredicate<CacheContinuousQueryEntry>() {
-                @Override public boolean apply(CacheContinuousQueryEntry entry) {
-                    return !entry.isFiltered();
+        if (!entries0.isEmpty()) {
+            Iterable<CacheEntryEvent<? extends K, ? extends V>> evts = F.viewReadOnly(entries0,
+                new C1<CacheContinuousQueryEntry, CacheEntryEvent<? extends K, ? extends V>>() {
+                    @Override public CacheEntryEvent<? extends K, ? extends V> apply(CacheContinuousQueryEntry e) {
+                        return new CacheContinuousQueryEvent<>(cache, cctx, e);
+                    }
+                },
+                new IgnitePredicate<CacheContinuousQueryEntry>() {
+                    @Override public boolean apply(CacheContinuousQueryEntry entry) {
+                        return !entry.isFiltered();
+                    }
                 }
-            }
-        );
+            );
 
-        locLsnr.onUpdated(evts);
+            locLsnr.onUpdated(evts);
+        }
     }
 
     /**
@@ -731,11 +710,11 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
          * @param initCntr Update counters.
          */
         public PartitionRecovery(IgniteLogger log, AffinityTopologyVersion topVer, @Nullable Long initCntr) {
-            assert topVer.topologyVersion() > 0 : topVer;
-
             this.log = log;
 
             if (initCntr != null) {
+                assert topVer.topologyVersion() > 0 : topVer;
+
                 this.lastFiredEvt = initCntr;
 
                 curTop = topVer;
@@ -878,32 +857,6 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
         }
 
         /**
-         * @param e Entry.
-         * @return Continuous query entry.
-         */
-        private CacheContinuousQueryEntry skipEntry(CacheContinuousQueryEntry e) {
-            if (lastFiredCntr.get() > e.updateCounter() || e.updateCounter() == 1L) {
-                e.markFiltered();
-
-                return e;
-            }
-            else {
-                buf.add(e.updateCounter());
-
-                // Double check. If another thread sent a event with counter higher than this event.
-                if (lastFiredCntr.get() > e.updateCounter() && buf.contains(e.updateCounter())) {
-                    buf.remove(e.updateCounter());
-
-                    e.markFiltered();
-
-                    return e;
-                }
-                else
-                    return null;
-            }
-        }
-
-        /**
          * Add continuous entry.
          *
          * @param e Cache continuous query entry.

http://git-wip-us.apache.org/repos/asf/ignite/blob/cb35e1d7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java
index dce04de..83ff32c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java
@@ -67,8 +67,9 @@ public interface CacheContinuousQueryListener<K, V> {
     /**
      * @param evt Event
      * @param topVer Topology version.
+     * @param primary Primary
      */
-    public void skipUpdateEvent(CacheContinuousQueryEvent<K, V> evt, AffinityTopologyVersion topVer);
+    public void skipUpdateEvent(CacheContinuousQueryEvent<K, V> evt, AffinityTopologyVersion topVer, boolean primary);
 
     /**
      * @param part Partition.

http://git-wip-us.apache.org/repos/asf/ignite/blob/cb35e1d7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
index 968fc23..840a61b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
@@ -166,7 +166,27 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
      * @param topVer Topology version.
      */
     public void skipUpdateEvent(Map<UUID, CacheContinuousQueryListener> lsnrs,
-        KeyCacheObject key, int partId, long updCntr, AffinityTopologyVersion topVer) {
+        KeyCacheObject key,
+        int partId,
+        long updCntr,
+        AffinityTopologyVersion topVer) {
+        skipUpdateEvent(lsnrs, key, partId, updCntr, true, topVer);
+    }
+
+    /**
+     * @param lsnrs Listeners to notify.
+     * @param key Entry key.
+     * @param partId Partition id.
+     * @param updCntr Updated counter.
+     * @param topVer Topology version.
+     * @param primary Primary.
+     */
+    public void skipUpdateEvent(Map<UUID, CacheContinuousQueryListener> lsnrs,
+        KeyCacheObject key,
+        int partId,
+        long updCntr,
+        boolean primary,
+        AffinityTopologyVersion topVer) {
         assert lsnrs != null;
 
         for (CacheContinuousQueryListener lsnr : lsnrs.values()) {
@@ -184,7 +204,7 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
             CacheContinuousQueryEvent evt = new CacheContinuousQueryEvent<>(
                 cctx.kernalContext().cache().jcache(cctx.name()), cctx, e0);
 
-            lsnr.skipUpdateEvent(evt, topVer);
+            lsnr.skipUpdateEvent(evt, topVer, primary);
         }
     }
 
@@ -281,7 +301,7 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
         boolean hasOldVal = oldVal != null;
 
         if (!hasNewVal && !hasOldVal) {
-            skipUpdateEvent(lsnrCol, key, partId, updateCntr, topVer);
+            skipUpdateEvent(lsnrCol, key, partId, updateCntr, primary, topVer);
 
             return;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cb35e1d7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterAbstractTest.java
new file mode 100644
index 0000000..d8a5006
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterAbstractTest.java
@@ -0,0 +1,613 @@
+/*
+ * 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.query.continuous;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.Cache;
+import javax.cache.configuration.Factory;
+import javax.cache.event.CacheEntryEvent;
+import javax.cache.event.CacheEntryUpdatedListener;
+import javax.cache.integration.CacheWriterException;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.CacheQueryEntryEvent;
+import org.apache.ignite.cache.query.ContinuousQuery;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.cache.store.CacheStoreAdapter;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.util.typedef.PA;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.lang.IgniteBiInClosure;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+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 org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import org.jsr166.ConcurrentHashMap8;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.LOCAL;
+import static org.apache.ignite.cache.CacheRebalanceMode.ASYNC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ * Continuous queries counter tests.
+ */
+public abstract class CacheContinuousQueryCounterAbstractTest extends GridCommonAbstractTest
+    implements Serializable {
+    /** */
+    protected static final String CACHE_NAME = "test_cache";
+
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** Latch timeout. */
+    protected static final long LATCH_TIMEOUT = 5000;
+
+    /** */
+    private static final String NO_CACHE_GRID_NAME = "noCacheGrid";
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setPeerClassLoadingEnabled(peerClassLoadingEnabled());
+
+        if (gridName.equals(NO_CACHE_GRID_NAME))
+            cfg.setClientMode(true);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
+        return cfg;
+    }
+
+    /**
+     * @return Cache configuration.
+     */
+    @NotNull private CacheConfiguration cacheConfiguration() {
+        CacheConfiguration cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setName(CACHE_NAME);
+        cacheCfg.setCacheMode(cacheMode());
+        cacheCfg.setAtomicityMode(atomicityMode());
+        cacheCfg.setNearConfiguration(nearConfiguration());
+        cacheCfg.setRebalanceMode(ASYNC);
+        cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
+        cacheCfg.setCacheStoreFactory(new StoreFactory());
+        cacheCfg.setReadThrough(true);
+        cacheCfg.setWriteThrough(true);
+        cacheCfg.setLoadPreviousValue(true);
+
+        return cacheCfg;
+    }
+
+    /**
+     * @return Peer class loading enabled flag.
+     */
+    protected boolean peerClassLoadingEnabled() {
+        return true;
+    }
+
+    /**
+     * @return Distribution.
+     */
+    protected NearCacheConfiguration nearConfiguration() {
+        return new NearCacheConfiguration();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(gridCount());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        GridTestUtils.waitForCondition(new PA() {
+            @Override public boolean apply() {
+                for (int i = 0; i < gridCount(); i++) {
+                    if (grid(i).cluster().nodes().size() != gridCount())
+                        return false;
+                }
+
+                return true;
+            }
+        }, 3000);
+
+        for (int i = 0; i < gridCount(); i++)
+            grid(i).destroyCache(CACHE_NAME);
+
+        for (int i = 0; i < gridCount(); i++)
+            grid(i).getOrCreateCache(cacheConfiguration());
+    }
+
+    /**
+     * @return Cache mode.
+     */
+    protected abstract CacheMode cacheMode();
+
+    /**
+     * @return Atomicity mode.
+     */
+    protected CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL;
+    }
+
+    /**
+     * @return Grids count.
+     */
+    protected abstract int gridCount();
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAllEntries() throws Exception {
+        IgniteCache<Integer, Integer> cache = grid(0).cache(CACHE_NAME);
+
+        ContinuousQuery<Integer, Integer> qry = new ContinuousQuery<>();
+
+        final Map<Integer, List<T2<Integer, Long>>> map = new HashMap<>();
+        final CountDownLatch latch = new CountDownLatch(5);
+
+        qry.setLocalListener(new CacheEntryUpdatedListener<Integer, Integer>() {
+            @Override public void onUpdated(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
+                for (CacheEntryEvent<? extends Integer, ? extends Integer> e : evts) {
+                    synchronized (map) {
+                        List<T2<Integer, Long>> vals = map.get(e.getKey());
+
+                        if (vals == null) {
+                            vals = new ArrayList<>();
+
+                            map.put(e.getKey(), vals);
+                        }
+
+                        vals.add(new T2<>(e.getValue(), e
+                            .unwrap(CacheQueryEntryEvent.class).getPartitionUpdateCounter()));
+                    }
+
+                    latch.countDown();
+                }
+            }
+        });
+
+        try (QueryCursor<Cache.Entry<Integer, Integer>> ignored = cache.query(qry)) {
+            cache.put(1, 1);
+            cache.put(2, 2);
+            cache.put(3, 3);
+
+            cache.remove(2);
+
+            cache.put(1, 10);
+
+            assert latch.await(LATCH_TIMEOUT, MILLISECONDS);
+
+            assertEquals(3, map.size());
+
+            List<T2<Integer, Long>> vals = map.get(1);
+
+            assertNotNull(vals);
+            assertEquals(2, vals.size());
+            assertEquals(1, (int)vals.get(0).get1());
+            assertEquals(1L, (long)vals.get(0).get2());
+            assertEquals(10, (int)vals.get(1).get1());
+            assertEquals(2L, (long)vals.get(1).get2());
+
+            vals = map.get(2);
+
+            assertNotNull(vals);
+            assertEquals(2, vals.size());
+            assertEquals(2, (int)vals.get(0).get1());
+            assertEquals(1L, (long)vals.get(0).get2());
+            assertNull(vals.get(1).get1());
+
+            vals = map.get(3);
+
+            assertNotNull(vals);
+            assertEquals(1, vals.size());
+            assertEquals(3, (int)vals.get(0).get1());
+            assertEquals(1L, (long)vals.get(0).get2());
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTwoQueryListener() throws Exception {
+        if (cacheMode() == LOCAL)
+            return;
+
+        final IgniteCache<Integer, Integer> cache = grid(0).cache(CACHE_NAME);
+        final IgniteCache<Integer, Integer> cache1 = grid(1).cache(CACHE_NAME);
+
+        final AtomicInteger cntr = new AtomicInteger(0);
+        final AtomicInteger cntr1 = new AtomicInteger(0);
+
+        final ContinuousQuery<Integer, Integer> qry1 = new ContinuousQuery<>();
+        final ContinuousQuery<Integer, Integer> qry2 = new ContinuousQuery<>();
+
+        final Map<Integer, List<T2<Integer, Long>>> map1 = new HashMap<>();
+        final Map<Integer, List<T2<Integer, Long>>> map2 = new HashMap<>();
+
+        qry1.setLocalListener(new CacheEntryUpdatedListener<Integer, Integer>() {
+            @Override public void onUpdated(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
+                for (CacheEntryEvent<? extends Integer, ? extends Integer> e : evts) {
+                    cntr.incrementAndGet();
+
+                    synchronized (map1) {
+                        List<T2<Integer, Long>> vals = map1.get(e.getKey());
+
+                        if (vals == null) {
+                            vals = new ArrayList<>();
+
+                            map1.put(e.getKey(), vals);
+                        }
+
+                        vals.add(new T2<>(e.getValue(),
+                            e.unwrap(CacheQueryEntryEvent.class).getPartitionUpdateCounter()));
+                    }
+                }
+            }
+        });
+
+        qry2.setLocalListener(new CacheEntryUpdatedListener<Integer, Integer>() {
+            @Override public void onUpdated(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
+                for (CacheEntryEvent<? extends Integer, ? extends Integer> e : evts) {
+                    cntr1.incrementAndGet();
+
+                    synchronized (map2) {
+                        List<T2<Integer, Long>> vals = map2.get(e.getKey());
+
+                        if (vals == null) {
+                            vals = new ArrayList<>();
+
+                            map2.put(e.getKey(), vals);
+                        }
+
+                        vals.add(new T2<>(e.getValue(),
+                            e.unwrap(CacheQueryEntryEvent.class).getPartitionUpdateCounter()));
+                    }
+                }
+            }
+        });
+
+        try (QueryCursor<Cache.Entry<Integer, Integer>> query2 = cache1.query(qry2);
+            QueryCursor<Cache.Entry<Integer, Integer>> query1 = cache.query(qry1)) {
+            for (int i = 0; i < gridCount(); i++) {
+                IgniteCache<Object, Object> cache0 = grid(i).cache(CACHE_NAME);
+
+                cache0.put(1, 1);
+                cache0.put(2, 2);
+                cache0.put(3, 3);
+
+                cache0.remove(1);
+                cache0.remove(2);
+                cache0.remove(3);
+
+                final int iter = i + 1;
+
+                assert GridTestUtils.waitForCondition(new PA() {
+                    @Override public boolean apply() {
+                        return iter * 6 /* count operation */ * 2 /* count continues queries*/
+                            == (cntr.get() + cntr1.get());
+                    }
+                }, 5000L);
+
+                checkEvents(map1, i);
+
+                map1.clear();
+
+                checkEvents(map2, i);
+
+                map2.clear();
+            }
+        }
+    }
+
+    /**
+     * @param evnts Events.
+     * @param iter Iteration.
+     */
+    private void checkEvents(Map<Integer, List<T2<Integer, Long>>> evnts, long iter) {
+        List<T2<Integer, Long>> val = evnts.get(1);
+
+        assertEquals(val.size(), 2);
+
+        // Check put 1
+        assertEquals(iter * 2 + 1, (long)val.get(0).get2());
+        assertEquals(1L, (long)val.get(0).get1());
+
+        // Check remove 1
+        assertEquals(iter * 2 + 2, (long)val.get(1).get2());
+        assertNull(val.get(1).get1());
+
+        val = evnts.get(2);
+
+        assertEquals(val.size(), 2);
+
+        // Check put 2
+        assertEquals(iter * 2 + 1, (long)val.get(0).get2());
+        assertEquals(2L, (long)val.get(0).get1());
+
+        // Check remove 2
+        assertEquals(iter * 2 + 2, (long)val.get(1).get2());
+        assertNull(val.get(1).get1());
+
+        val = evnts.get(3);
+
+        assertEquals(val.size(), 2);
+
+        // Check put 3
+        assertEquals(iter * 2 + 1, (long)val.get(0).get2());
+        assertEquals(3L, (long)val.get(0).get1());
+
+        // Check remove 3
+        assertEquals(iter * 2 + 2, (long)val.get(1).get2());
+        assertNull(val.get(1).get1());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRestartQuery() throws Exception {
+        IgniteCache<Integer, Integer> cache = grid(0).cache(CACHE_NAME);
+
+        final int keyCnt = 300;
+
+        final int updateKey = 1;
+
+        for (int i = 0; i < keyCnt; i++)
+            cache.put(updateKey, i);
+
+        for (int i = 0; i < 10; i++) {
+            if (i % 2 == 0) {
+                final AtomicInteger cntr = new AtomicInteger(0);
+
+                ContinuousQuery<Integer, Integer> qry = new ContinuousQuery<>();
+
+                final List<T2<Integer, Long>> vals = new ArrayList<>();
+
+                qry.setLocalListener(new CacheEntryUpdatedListener<Integer, Integer>() {
+                    @Override public void onUpdated(
+                        Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
+                        for (CacheEntryEvent<? extends Integer, ? extends Integer> e : evts) {
+                            cntr.incrementAndGet();
+
+                            synchronized (vals) {
+                                vals.add(new T2<>(e.getValue(),
+                                    e.unwrap(CacheQueryEntryEvent.class).getPartitionUpdateCounter()));
+                            }
+                        }
+                    }
+                });
+
+                try (QueryCursor<Cache.Entry<Integer, Integer>> ignore = cache.query(qry)) {
+                    for (int key = 0; key < keyCnt; key++)
+                        cache.put(updateKey, cache.get(updateKey) + 1);
+
+                    assert GridTestUtils.waitForCondition(new PA() {
+                        @Override public boolean apply() {
+                            return cntr.get() == keyCnt;
+                        }
+                    }, 2000L);
+
+                    for (T2<Integer, Long> val : vals) {
+                        assertEquals(vals.size(), keyCnt);
+
+                        assertEquals((long)val.get1() + 1, (long)val.get2());
+                    }
+                }
+            }
+            else {
+                for (int key = 0; key < keyCnt; key++)
+                    cache.put(updateKey, cache.get(updateKey) + 1);
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEntriesByFilter() throws Exception {
+        IgniteCache<Integer, Integer> cache = grid(0).cache(CACHE_NAME);
+
+        ContinuousQuery<Integer, Integer> qry = new ContinuousQuery<>();
+
+        final Map<Integer, List<T2<Integer, Long>>> map = new HashMap<>();
+        final CountDownLatch latch = new CountDownLatch(8);
+
+        qry.setLocalListener(new CacheEntryUpdatedListener<Integer, Integer>() {
+            @Override public void onUpdated(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
+                for (CacheEntryEvent<? extends Integer, ? extends Integer> e : evts) {
+                    synchronized (map) {
+                        List<T2<Integer, Long>> vals = map.get(e.getKey());
+
+                        if (vals == null) {
+                            vals = new ArrayList<>();
+
+                            map.put(e.getKey(), vals);
+                        }
+
+                        vals.add(new T2<>(e.getValue(),
+                            e.unwrap(CacheQueryEntryEvent.class).getPartitionUpdateCounter()));
+                    }
+
+                    latch.countDown();
+                }
+            }
+        });
+
+        qry.setRemoteFilter(new CacheEntryEventSerializableFilter<Integer,Integer>() {
+            @Override public boolean evaluate(CacheEntryEvent<? extends Integer,? extends Integer> evt) {
+                return evt.getValue() % 2 == 0;
+            }
+        });
+
+        try (QueryCursor<Cache.Entry<Integer, Integer>> ignored = cache.query(qry)) {
+            cache.put(1, 1);
+            cache.put(1, 2);
+            cache.put(1, 3);
+            cache.put(1, 4);
+
+            cache.put(2, 1);
+            cache.put(2, 2);
+            cache.put(2, 3);
+            cache.put(2, 4);
+
+            cache.remove(1);
+            cache.remove(2);
+
+            cache.put(1, 10);
+            cache.put(2, 40);
+
+            assert latch.await(LATCH_TIMEOUT, MILLISECONDS);
+
+            assertEquals(2, map.size());
+
+            List<T2<Integer, Long>> vals = map.get(1);
+
+            assertNotNull(vals);
+            assertEquals(4, vals.size());
+
+            assertEquals((int)vals.get(0).get1(), 2);
+            assertEquals((long)vals.get(0).get1(), (long)vals.get(0).get2());
+
+            assertEquals((int)vals.get(1).get1(), 4);
+            assertEquals((long)vals.get(1).get1(), (long)vals.get(1).get2());
+
+            assertNull(vals.get(2).get1());
+            assertEquals(5, (long)vals.get(2).get2());
+
+            assertEquals((int)vals.get(3).get1(), 10);
+            assertEquals(6, (long)vals.get(3).get2());
+
+            vals = map.get(2);
+
+            assertNotNull(vals);
+            assertEquals(4, vals.size());
+
+            assertEquals((int)vals.get(0).get1(), 2);
+            assertEquals((long)vals.get(0).get1(), (long)vals.get(0).get2());
+
+            assertEquals((int)vals.get(1).get1(), 4);
+            assertEquals((long)vals.get(1).get1(), (long)vals.get(1).get2());
+
+            assertNull(vals.get(2).get1());
+            assertEquals(5, (long)vals.get(2).get2());
+
+            assertEquals((int)vals.get(3).get1(), 40);
+            assertEquals(6, (long)vals.get(3).get2());
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLoadCache() throws Exception {
+        IgniteCache<Integer, Integer> cache = grid(0).cache(CACHE_NAME);
+
+        ContinuousQuery<Integer, Integer> qry = new ContinuousQuery<>();
+
+        final Map<Integer, T2<Integer, Long>> map = new ConcurrentHashMap8<>();
+        final CountDownLatch latch = new CountDownLatch(10);
+
+        qry.setLocalListener(new CacheEntryUpdatedListener<Integer, Integer>() {
+            @Override public void onUpdated(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
+                for (CacheEntryEvent<? extends Integer, ? extends Integer> e : evts) {
+                    map.put(e.getKey(), new T2<>(e.getValue(),
+                        e.unwrap(CacheQueryEntryEvent.class).getPartitionUpdateCounter()));
+
+                    latch.countDown();
+                }
+            }
+        });
+
+        try (QueryCursor<Cache.Entry<Integer, Integer>> ignored = cache.query(qry)) {
+            cache.loadCache(null, 0);
+
+            assert latch.await(LATCH_TIMEOUT, MILLISECONDS) : "Count: " + latch.getCount();
+
+            assertEquals(10, map.size());
+
+            for (int i = 0; i < 10; i++) {
+                assertEquals(i, (int)map.get(i).get1());
+                assertEquals((long)1, (long)map.get(i).get2());
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    private static class StoreFactory implements Factory<CacheStore> {
+        @Override public CacheStore create() {
+            return new TestStore();
+        }
+    }
+
+    /**
+     * Store.
+     */
+    private static class TestStore extends CacheStoreAdapter<Object, Object> {
+        /** {@inheritDoc} */
+        @Override public void loadCache(IgniteBiInClosure<Object, Object> clo, Object... args) {
+            for (int i = 0; i < 10; i++)
+                clo.apply(i, i);
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Object load(Object key) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<?, ?> entry) throws CacheWriterException {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) throws CacheWriterException {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cb35e1d7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterPartitionedAtomicTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterPartitionedAtomicTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterPartitionedAtomicTest.java
new file mode 100644
index 0000000..7b97928
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterPartitionedAtomicTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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.query.continuous;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+
+/**
+ *
+ */
+public class CacheContinuousQueryCounterPartitionedAtomicTest extends CacheContinuousQueryCounterAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.ATOMIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 2;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cb35e1d7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterPartitionedTxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterPartitionedTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterPartitionedTxTest.java
new file mode 100644
index 0000000..aa42832
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterPartitionedTxTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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.query.continuous;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+
+/**
+ *
+ */
+public class CacheContinuousQueryCounterPartitionedTxTest extends CacheContinuousQueryCounterAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 2;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cb35e1d7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterReplicatedAtomicTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterReplicatedAtomicTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterReplicatedAtomicTest.java
new file mode 100644
index 0000000..afa7a22
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterReplicatedAtomicTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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.query.continuous;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+
+/**
+ *
+ */
+public class CacheContinuousQueryCounterReplicatedAtomicTest extends CacheContinuousQueryCounterAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.REPLICATED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.ATOMIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 2;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cb35e1d7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterReplicatedTxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterReplicatedTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterReplicatedTxTest.java
new file mode 100644
index 0000000..4ee12de
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterReplicatedTxTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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.query.continuous;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+
+/**
+ *
+ */
+public class CacheContinuousQueryCounterReplicatedTxTest extends CacheContinuousQueryCounterAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 2;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cb35e1d7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryRandomOperationsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryRandomOperationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryRandomOperationsTest.java
index d9b2091..62ed66f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryRandomOperationsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryRandomOperationsTest.java
@@ -18,7 +18,14 @@
 package org.apache.ignite.internal.processors.cache.query.continuous;
 
 import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.Random;
+import java.util.SortedMap;
+import java.util.TreeMap;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
@@ -31,10 +38,13 @@ import javax.cache.integration.CacheLoaderException;
 import javax.cache.integration.CacheWriterException;
 import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.MutableEntry;
+import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.cache.query.CacheQueryEntryEvent;
 import org.apache.ignite.cache.query.ContinuousQuery;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.store.CacheStore;
@@ -46,6 +56,9 @@ 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.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
@@ -58,6 +71,12 @@ import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryRandomOperationsTest.ContinuousDeploy.ALL;
+import static org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryRandomOperationsTest.ContinuousDeploy.CLIENT;
+import static org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryRandomOperationsTest.ContinuousDeploy.SERVER;
+import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE;
 
 /**
  *
@@ -70,12 +89,15 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
     private static final int NODES = 5;
 
     /** */
-    private static final int KEYS = 10;
+    private static final int KEYS = 50;
 
     /** */
     private static final int VALS = 10;
 
     /** */
+    public static final int ITERATION_CNT = 100;
+
+    /** */
     private boolean client;
 
     /** {@inheritDoc} */
@@ -110,6 +132,19 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
     /**
      * @throws Exception If failed.
      */
+    public void testAtomicClient() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            ATOMIC,
+            ONHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg, CLIENT);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testAtomic() throws Exception {
         CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
             1,
@@ -117,7 +152,20 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
             ONHEAP_TIERED,
             false);
 
-        testContinuousQuery(ccfg);
+        testContinuousQuery(ccfg, SERVER);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicAllNodes() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            ATOMIC,
+            ONHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg, ALL);
     }
 
     /**
@@ -130,7 +178,33 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
             ONHEAP_TIERED,
             false);
 
-        testContinuousQuery(ccfg);
+        testContinuousQuery(ccfg, SERVER);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicReplicatedAllNodes() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED,
+            0,
+            ATOMIC,
+            ONHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg, ALL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicReplicatedClient() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED,
+            0,
+            ATOMIC,
+            ONHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg, CLIENT);
     }
 
     /**
@@ -143,7 +217,33 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
             OFFHEAP_VALUES,
             false);
 
-        testContinuousQuery(ccfg);
+        testContinuousQuery(ccfg, SERVER);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicOffheapValuesAllNodes() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            ATOMIC,
+            OFFHEAP_VALUES,
+            false);
+
+        testContinuousQuery(ccfg, ALL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicOffheapValuesClient() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            ATOMIC,
+            OFFHEAP_VALUES,
+            false);
+
+        testContinuousQuery(ccfg, CLIENT);
     }
 
     /**
@@ -156,7 +256,33 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
             OFFHEAP_TIERED,
             false);
 
-        testContinuousQuery(ccfg);
+        testContinuousQuery(ccfg, SERVER);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicOffheapTieredAllNodes() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            ATOMIC,
+            OFFHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg, ALL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicOffheapTieredClient() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            ATOMIC,
+            OFFHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg, CLIENT);
     }
 
     /**
@@ -169,7 +295,33 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
             ONHEAP_TIERED,
             false);
 
-        testContinuousQuery(ccfg);
+        testContinuousQuery(ccfg, SERVER);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicNoBackupsAllNodes() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            0,
+            ATOMIC,
+            ONHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg, ALL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicNoBackupsClient() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            0,
+            ATOMIC,
+            ONHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg, CLIENT);
     }
 
     /**
@@ -182,7 +334,59 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
             ONHEAP_TIERED,
             false);
 
-        testContinuousQuery(ccfg);
+        testContinuousQuery(ccfg, SERVER);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxAllNodes() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL,
+            ONHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg, ALL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxExplicit() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL,
+            ONHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg, SERVER);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxClient() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL,
+            ONHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg, CLIENT);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxClientExplicit() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL,
+            ONHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg, CLIENT);
     }
 
     /**
@@ -195,7 +399,20 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
             ONHEAP_TIERED,
             false);
 
-        testContinuousQuery(ccfg);
+        testContinuousQuery(ccfg, SERVER);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxReplicatedClient() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED,
+            0,
+            TRANSACTIONAL,
+            ONHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg, CLIENT);
     }
 
     /**
@@ -208,7 +425,46 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
             OFFHEAP_VALUES,
             false);
 
-        testContinuousQuery(ccfg);
+        testContinuousQuery(ccfg, SERVER);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxOffheapValuesAllNodes() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL,
+            OFFHEAP_VALUES,
+            false);
+
+        testContinuousQuery(ccfg, ALL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxOffheapValuesExplicit() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL,
+            OFFHEAP_VALUES,
+            false);
+
+        testContinuousQuery(ccfg, SERVER);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxOffheapValuesClient() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL,
+            OFFHEAP_VALUES,
+            false);
+
+        testContinuousQuery(ccfg, CLIENT);
     }
 
     /**
@@ -221,7 +477,46 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
             OFFHEAP_TIERED,
             false);
 
-        testContinuousQuery(ccfg);
+        testContinuousQuery(ccfg, SERVER);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxOffheapTieredAllNodes() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL,
+            OFFHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg, ALL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxOffheapTieredClient() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL,
+            OFFHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg, CLIENT);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxOffheapTieredClientExplicit() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL,
+            OFFHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg, CLIENT);
     }
 
     /**
@@ -234,54 +529,141 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
             ONHEAP_TIERED,
             false);
 
-        testContinuousQuery(ccfg);
+        testContinuousQuery(ccfg, SERVER);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxNoBackupsAllNodes() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            0,
+            TRANSACTIONAL,
+            ONHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg, ALL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxNoBackupsExplicit() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            0,
+            TRANSACTIONAL,
+            ONHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg, SERVER);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxNoBackupsClient() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            0,
+            TRANSACTIONAL,
+            ONHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg, CLIENT);
     }
 
     /**
      * @param ccfg Cache configuration.
+     * @param deploy The place where continuous query will be started.
      * @throws Exception If failed.
      */
-    private void testContinuousQuery(CacheConfiguration<Object, Object> ccfg) throws Exception {
+    private void testContinuousQuery(CacheConfiguration<Object, Object> ccfg, ContinuousDeploy deploy)
+        throws Exception {
         ignite(0).createCache(ccfg);
 
         try {
-            IgniteCache<Object, Object> cache = ignite(NODES - 1).cache(ccfg.getName());
-
             long seed = System.currentTimeMillis();
 
             Random rnd = new Random(seed);
 
             log.info("Random seed: " + seed);
 
-            ContinuousQuery<Object, Object> qry = new ContinuousQuery<>();
+            List<BlockingQueue<CacheEntryEvent<?, ?>>> evtsQueues = new ArrayList<>();
 
-            final BlockingQueue<CacheEntryEvent<?, ?>> evtsQueue =
-                new ArrayBlockingQueue<>(10_000);
+            Collection<QueryCursor<?>> curs = new ArrayList<>();
 
-            qry.setLocalListener(new CacheEntryUpdatedListener<Object, Object>() {
-                @Override public void onUpdated(Iterable<CacheEntryEvent<?, ?>> evts) {
-                    for (CacheEntryEvent<?, ?> evt : evts) {
-                        // System.out.println("Event: " + evt);
+            if (deploy == CLIENT) {
+                ContinuousQuery<Object, Object> qry = new ContinuousQuery<>();
 
-                        evtsQueue.add(evt);
+                final BlockingQueue<CacheEntryEvent<?, ?>> evtsQueue = new ArrayBlockingQueue<>(50_000);
+
+                qry.setLocalListener(new CacheEntryUpdatedListener<Object, Object>() {
+                    @Override public void onUpdated(Iterable<CacheEntryEvent<?, ?>> evts) {
+                        for (CacheEntryEvent<?, ?> evt : evts)
+                            evtsQueue.add(evt);
                     }
-                }
-            });
+                });
+
+                evtsQueues.add(evtsQueue);
+
+                QueryCursor<?> cur = grid(NODES - 1).cache(ccfg.getName()).query(qry);
+
+                curs.add(cur);
+            }
+            else if (deploy == SERVER) {
+                ContinuousQuery<Object, Object> qry = new ContinuousQuery<>();
+
+                final BlockingQueue<CacheEntryEvent<?, ?>> evtsQueue = new ArrayBlockingQueue<>(50_000);
+
+                qry.setLocalListener(new CacheEntryUpdatedListener<Object, Object>() {
+                    @Override public void onUpdated(Iterable<CacheEntryEvent<?, ?>> evts) {
+                        for (CacheEntryEvent<?, ?> evt : evts)
+                            evtsQueue.add(evt);
+                    }
+                });
+
+                evtsQueues.add(evtsQueue);
+
+                QueryCursor<?> cur = grid(rnd.nextInt(NODES - 1)).cache(ccfg.getName()).query(qry);
+
+                curs.add(cur);
+            }
+            else {
+                for (int i = 0; i < NODES - 1; i++) {
+                    ContinuousQuery<Object, Object> qry = new ContinuousQuery<>();
+
+                    final BlockingQueue<CacheEntryEvent<?, ?>> evtsQueue = new ArrayBlockingQueue<>(50_000);
+
+                    qry.setLocalListener(new CacheEntryUpdatedListener<Object, Object>() {
+                        @Override public void onUpdated(Iterable<CacheEntryEvent<?, ?>> evts) {
+                            for (CacheEntryEvent<?, ?> evt : evts)
+                                evtsQueue.add(evt);
+                        }
+                    });
+
+                    evtsQueues.add(evtsQueue);
 
-            QueryCursor<?> cur = cache.query(qry);
+                    QueryCursor<?> cur = ignite(i).cache(ccfg.getName()).query(qry);
+
+                    curs.add(cur);
+                }
+            }
 
             ConcurrentMap<Object, Object> expData = new ConcurrentHashMap<>();
 
+            Map<Integer, Long> partCntr = new ConcurrentHashMap<>();
+
             try {
-                for (int i = 0; i < 1000; i++) {
-                    if (i % 100 == 0)
+                for (int i = 0; i < ITERATION_CNT; i++) {
+                    if (i % 20 == 0)
                         log.info("Iteration: " + i);
 
-                    randomUpdate(rnd, evtsQueue, expData, cache);
+                    for (int idx = 0; idx < NODES; idx++)
+                        randomUpdate(rnd, evtsQueues, expData, partCntr, grid(idx).cache(ccfg.getName()));
                 }
             }
             finally {
-                cur.close();
+                for (QueryCursor<?> cur : curs)
+                    cur.close();
             }
         }
         finally {
@@ -291,176 +673,389 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
 
     /**
      * @param rnd Random generator.
-     * @param evtsQueue Events queue.
+     * @param evtsQueues Events queue.
      * @param expData Expected cache data.
+     * @param partCntr Partition counter.
      * @param cache Cache.
      * @throws Exception If failed.
      */
     private void randomUpdate(
         Random rnd,
-        BlockingQueue<CacheEntryEvent<?, ?>> evtsQueue,
+        List<BlockingQueue<CacheEntryEvent<?, ?>>> evtsQueues,
         ConcurrentMap<Object, Object> expData,
+        Map<Integer, Long> partCntr,
         IgniteCache<Object, Object> cache)
         throws Exception {
         Object key = new QueryTestKey(rnd.nextInt(KEYS));
         Object newVal = value(rnd);
         Object oldVal = expData.get(key);
 
-        int op = rnd.nextInt(11);
+        int op = rnd.nextInt(13);
 
-        // log.info("Random operation [key=" + key + ", op=" + op + ']');
+        Ignite ignite = cache.unwrap(Ignite.class);
 
-        switch (op) {
-            case 0: {
-                cache.put(key, newVal);
+        Transaction tx = null;
 
-                waitEvent(evtsQueue, key, newVal, oldVal);
+        if (cache.getConfiguration(CacheConfiguration.class).getAtomicityMode() == TRANSACTIONAL && rnd.nextBoolean())
+            tx = ignite.transactions().txStart(txRandomConcurrency(rnd), txRandomIsolation(rnd));
 
-                expData.put(key, newVal);
+        try {
+            // log.info("Random operation [key=" + key + ", op=" + op + ']');
 
-                break;
-            }
+            switch (op) {
+                case 0: {
+                    cache.put(key, newVal);
 
-            case 1: {
-                cache.getAndPut(key, newVal);
+                    if (tx != null)
+                        tx.commit();
 
-                waitEvent(evtsQueue, key, newVal, oldVal);
+                    updatePartitionCounter(cache, key, partCntr);
 
-                expData.put(key, newVal);
+                    waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, newVal, oldVal);
 
-                break;
-            }
+                    expData.put(key, newVal);
 
-            case 2: {
-                cache.remove(key);
+                    break;
+                }
 
-                waitEvent(evtsQueue, key, null, oldVal);
+                case 1: {
+                    cache.getAndPut(key, newVal);
 
-                expData.remove(key);
+                    if (tx != null)
+                        tx.commit();
 
-                break;
-            }
+                    updatePartitionCounter(cache, key, partCntr);
 
-            case 3: {
-                cache.getAndRemove(key);
+                    waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, newVal, oldVal);
 
-                waitEvent(evtsQueue, key, null, oldVal);
+                    expData.put(key, newVal);
 
-                expData.remove(key);
+                    break;
+                }
 
-                break;
-            }
+                case 2: {
+                    cache.remove(key);
 
-            case 4: {
-                cache.invoke(key, new EntrySetValueProcessor(newVal, rnd.nextBoolean()));
+                    if (tx != null)
+                        tx.commit();
 
-                waitEvent(evtsQueue, key, newVal, oldVal);
+                    updatePartitionCounter(cache, key, partCntr);
 
-                expData.put(key, newVal);
+                    waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, null, oldVal);
 
-                break;
-            }
+                    expData.remove(key);
 
-            case 5: {
-                cache.invoke(key, new EntrySetValueProcessor(null, rnd.nextBoolean()));
+                    break;
+                }
 
-                waitEvent(evtsQueue, key, null, oldVal);
+                case 3: {
+                    cache.getAndRemove(key);
 
-                expData.remove(key);
+                    if (tx != null)
+                        tx.commit();
 
-                break;
-            }
+                    updatePartitionCounter(cache, key, partCntr);
 
-            case 6: {
-                cache.putIfAbsent(key, newVal);
+                    waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, null, oldVal);
 
-                if (oldVal == null) {
-                    waitEvent(evtsQueue, key, newVal, null);
+                    expData.remove(key);
 
-                    expData.put(key, newVal);
+                    break;
                 }
-                else
-                    checkNoEvent(evtsQueue);
 
-                break;
-            }
+                case 4: {
+                    cache.invoke(key, new EntrySetValueProcessor(newVal, rnd.nextBoolean()));
+
+                    if (tx != null)
+                        tx.commit();
 
-            case 7: {
-                cache.getAndPutIfAbsent(key, newVal);
+                    updatePartitionCounter(cache, key, partCntr);
 
-                if (oldVal == null) {
-                    waitEvent(evtsQueue, key, newVal, null);
+                    waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, newVal, oldVal);
 
                     expData.put(key, newVal);
+
+                    break;
                 }
-                else
-                    checkNoEvent(evtsQueue);
 
-                break;
-            }
+                case 5: {
+                    cache.invoke(key, new EntrySetValueProcessor(null, rnd.nextBoolean()));
 
-            case 8: {
-                cache.replace(key, newVal);
+                    if (tx != null)
+                        tx.commit();
 
-                if (oldVal != null) {
-                    waitEvent(evtsQueue, key, newVal, oldVal);
+                    updatePartitionCounter(cache, key, partCntr);
 
-                    expData.put(key, newVal);
+                    waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, null, oldVal);
+
+                    expData.remove(key);
+
+                    break;
                 }
-                else
-                    checkNoEvent(evtsQueue);
 
-                break;
-            }
+                case 6: {
+                    cache.putIfAbsent(key, newVal);
 
-            case 9: {
-                cache.getAndReplace(key, newVal);
+                    if (tx != null)
+                        tx.commit();
 
-                if (oldVal != null) {
-                    waitEvent(evtsQueue, key, newVal, oldVal);
+                    if (oldVal == null) {
+                        updatePartitionCounter(cache, key, partCntr);
 
-                    expData.put(key, newVal);
+                        waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, newVal, null);
+
+                        expData.put(key, newVal);
+                    }
+                    else
+                        checkNoEvent(evtsQueues);
+
+                    break;
                 }
-                else
-                    checkNoEvent(evtsQueue);
 
-                break;
-            }
+                case 7: {
+                    cache.getAndPutIfAbsent(key, newVal);
+
+                    if (tx != null)
+                        tx.commit();
+
+                    if (oldVal == null) {
+                        updatePartitionCounter(cache, key, partCntr);
+
+                        waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, newVal, null);
+
+                        expData.put(key, newVal);
+                    }
+                    else
+                        checkNoEvent(evtsQueues);
+
+                    break;
+                }
 
-            case 10: {
-                if (oldVal != null) {
-                    Object replaceVal = value(rnd);
+                case 8: {
+                    cache.replace(key, newVal);
 
-                    boolean success = replaceVal.equals(oldVal);
+                    if (tx != null)
+                        tx.commit();
 
-                    if (success) {
-                        cache.replace(key, replaceVal, newVal);
+                    if (oldVal != null) {
+                        updatePartitionCounter(cache, key, partCntr);
 
-                        waitEvent(evtsQueue, key, newVal, oldVal);
+                        waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, newVal, oldVal);
 
                         expData.put(key, newVal);
                     }
+                    else
+                        checkNoEvent(evtsQueues);
+
+                    break;
+                }
+
+                case 9: {
+                    cache.getAndReplace(key, newVal);
+
+                    if (tx != null)
+                        tx.commit();
+
+                    if (oldVal != null) {
+                        updatePartitionCounter(cache, key, partCntr);
+
+                        waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, newVal, oldVal);
+
+                        expData.put(key, newVal);
+                    }
+                    else
+                        checkNoEvent(evtsQueues);
+
+                    break;
+                }
+
+                case 10: {
+                    if (oldVal != null) {
+                        Object replaceVal = value(rnd);
+
+                        boolean success = replaceVal.equals(oldVal);
+
+                        if (success) {
+                            cache.replace(key, replaceVal, newVal);
+
+                            if (tx != null)
+                                tx.commit();
+
+                            updatePartitionCounter(cache, key, partCntr);
+
+                            waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, newVal, oldVal);
+
+                            expData.put(key, newVal);
+                        }
+                        else {
+                            cache.replace(key, replaceVal, newVal);
+
+                            if (tx != null)
+                                tx.commit();
+
+                            checkNoEvent(evtsQueues);
+                        }
+                    }
                     else {
-                        cache.replace(key, replaceVal, newVal);
+                        cache.replace(key, value(rnd), newVal);
+
+                        if (tx != null)
+                            tx.commit();
 
-                        checkNoEvent(evtsQueue);
+                        checkNoEvent(evtsQueues);
                     }
+
+                    break;
                 }
-                else {
-                    cache.replace(key, value(rnd), newVal);
 
-                    checkNoEvent(evtsQueue);
+                case 11: {
+                    SortedMap<Object, Object> vals = new TreeMap<>();
+
+                    while (vals.size() < KEYS / 5)
+                        vals.put(new QueryTestKey(rnd.nextInt(KEYS)), value(rnd));
+
+                    cache.putAll(vals);
+
+                    if (tx != null)
+                        tx.commit();
+
+                    for (Map.Entry<Object, Object> e : vals.entrySet())
+                        updatePartitionCounter(cache, e.getKey(), partCntr);
+
+                    waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), vals, expData);
+
+                    expData.putAll(vals);
+
+                    break;
                 }
 
-                break;
+                case 12: {
+                    SortedMap<Object, Object> vals = new TreeMap<>();
+
+                    while (vals.size() < KEYS / 5)
+                        vals.put(new QueryTestKey(rnd.nextInt(KEYS)), newVal);
+
+                    cache.invokeAll(vals.keySet(), new EntrySetValueProcessor(newVal, rnd.nextBoolean()));
+
+                    if (tx != null)
+                        tx.commit();
+
+                    for (Map.Entry<Object, Object> e : vals.entrySet())
+                        updatePartitionCounter(cache, e.getKey(), partCntr);
+
+                    waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), vals, expData);
+
+                    for (Object o : vals.keySet())
+                        expData.put(o, newVal);
+
+                    break;
+                }
+
+                default:
+                    fail("Op:" + op);
             }
+        } finally {
+            if (tx != null)
+                tx.close();
+        }
+    }
+
+    /**
+     *  @param evtsQueues Queue.
+     * @param partCntrs Counters.
+     * @param aff Affinity.
+     * @param vals Values.
+     * @param expData Expected data.
+     */
+    private void waitAndCheckEvent(List<BlockingQueue<CacheEntryEvent<?, ?>>> evtsQueues,
+        Map<Integer, Long> partCntrs,
+        Affinity<Object> aff,
+        SortedMap<Object, Object> vals,
+        Map<Object, Object> expData)
+        throws Exception {
+        for (BlockingQueue<CacheEntryEvent<?, ?>> evtsQueue : evtsQueues) {
+            Map<Object, CacheEntryEvent> rcvEvts = new HashMap<>();
+
+            for (int i = 0; i < vals.size(); i++) {
+                CacheEntryEvent<?, ?> evt = evtsQueue.poll(5, SECONDS);
+
+                rcvEvts.put(evt.getKey(), evt);
+            }
+
+            assertEquals(vals.size(), rcvEvts.size());
+
+            for (Map.Entry<Object, Object> e : vals.entrySet()) {
+                Object key = e.getKey();
+                Object val = e.getValue();
+                Object oldVal = expData.get(key);
+
+                if (val == null && oldVal == null) {
+                    checkNoEvent(evtsQueues);
 
-            default:
-                fail();
+                    continue;
+                }
+
+                CacheEntryEvent evt = rcvEvts.get(key);
+
+                assertNotNull("Failed to wait for event [key=" + key + ", val=" + val + ", oldVal=" + oldVal + ']',
+                    evt);
+                assertEquals(key, evt.getKey());
+                assertEquals(val, evt.getValue());
+                assertEquals(oldVal, evt.getOldValue());
+
+                long cntr = partCntrs.get(aff.partition(key));
+                CacheQueryEntryEvent qryEntryEvt = (CacheQueryEntryEvent)evt.unwrap(CacheQueryEntryEvent.class);
+
+                assertNotNull(cntr);
+                assertNotNull(qryEntryEvt);
+
+                assertEquals(cntr, qryEntryEvt.getPartitionUpdateCounter());
+            }
         }
     }
 
     /**
+     * @param rnd {@link Random}.
+     * @return {@link TransactionIsolation}.
+     */
+    private TransactionIsolation txRandomIsolation(Random rnd) {
+        int val = rnd.nextInt(3);
+
+        if (val == 0)
+            return READ_COMMITTED;
+        else if (val == 1)
+            return REPEATABLE_READ;
+        else
+            return SERIALIZABLE;
+    }
+
+    /**
+     * @param rnd {@link Random}.
+     * @return {@link TransactionConcurrency}.
+     */
+    private TransactionConcurrency txRandomConcurrency(Random rnd) {
+        return rnd.nextBoolean() ? TransactionConcurrency.OPTIMISTIC : TransactionConcurrency.PESSIMISTIC;
+    }
+
+    /**
+     * @param cache Cache.
+     * @param key Key
+     * @param cntrs Partition counters.
+     */
+    private void updatePartitionCounter(IgniteCache<Object, Object> cache, Object key, Map<Integer, Long> cntrs) {
+        Affinity<Object> aff = cache.unwrap(Ignite.class).affinity(cache.getName());
+
+        int part = aff.partition(key);
+
+        Long partCntr = cntrs.get(part);
+
+        if (partCntr == null)
+            partCntr = 0L;
+
+        cntrs.put(part, ++partCntr);
+    }
+
+    /**
      * @param rnd Random generator.
      * @return Cache value.
      */
@@ -469,38 +1064,55 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
     }
 
     /**
-     * @param evtsQueue Event queue.
+     * @param evtsQueues Event queue.
+     * @param partCntrs Partition counters.
+     * @param aff Affinity function.
      * @param key Key.
      * @param val Value.
      * @param oldVal Old value.
      * @throws Exception If failed.
      */
-    private void waitEvent(BlockingQueue<CacheEntryEvent<?, ?>> evtsQueue,
-        Object key, Object val, Object oldVal) throws Exception {
+    private void waitAndCheckEvent(List<BlockingQueue<CacheEntryEvent<?, ?>>> evtsQueues,
+        Map<Integer, Long> partCntrs,
+        Affinity<Object> aff,
+        Object key,
+        Object val,
+        Object oldVal)
+        throws Exception {
         if (val == null && oldVal == null) {
-            checkNoEvent(evtsQueue);
+            checkNoEvent(evtsQueues);
 
             return;
         }
 
-        CacheEntryEvent<?, ?> evt = evtsQueue.poll(5, SECONDS);
+        for (BlockingQueue<CacheEntryEvent<?, ?>> evtsQueue : evtsQueues) {
+            CacheEntryEvent<?, ?> evt = evtsQueue.poll(5, SECONDS);
+
+            assertNotNull("Failed to wait for event [key=" + key + ", val=" + val + ", oldVal=" + oldVal + ']', evt);
+            assertEquals(key, evt.getKey());
+            assertEquals(val, evt.getValue());
+            assertEquals(oldVal, evt.getOldValue());
 
-        assertNotNull("Failed to wait for event [key=" + key +
-            ", val=" + val +
-            ", oldVal=" + oldVal + ']', evt);
-        assertEquals(key, evt.getKey());
-        assertEquals(val, evt.getValue());
-        assertEquals(oldVal, evt.getOldValue());
+            long cntr = partCntrs.get(aff.partition(key));
+            CacheQueryEntryEvent qryEntryEvt = evt.unwrap(CacheQueryEntryEvent.class);
+
+            assertNotNull(cntr);
+            assertNotNull(qryEntryEvt);
+
+            assertEquals(cntr, qryEntryEvt.getPartitionUpdateCounter());
+        }
     }
 
     /**
-     * @param evtsQueue Event queue.
+     * @param evtsQueues Event queue.
      * @throws Exception If failed.
      */
-    private void checkNoEvent(BlockingQueue<CacheEntryEvent<?, ?>> evtsQueue) throws Exception {
-        CacheEntryEvent<?, ?> evt = evtsQueue.poll(50, MILLISECONDS);
+    private void checkNoEvent(List<BlockingQueue<CacheEntryEvent<?, ?>>> evtsQueues) throws Exception {
+        for (BlockingQueue<CacheEntryEvent<?, ?>> evtsQueue : evtsQueues) {
+            CacheEntryEvent<?, ?> evt = evtsQueue.poll(50, MILLISECONDS);
 
-        assertNull(evt);
+            assertNull(evt);
+        }
     }
 
     /**
@@ -564,7 +1176,7 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
     /**
      *
      */
-    static class QueryTestKey implements Serializable {
+    static class QueryTestKey implements Serializable, Comparable {
         /** */
         private final Integer key;
 
@@ -597,6 +1209,11 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
         @Override public String toString() {
             return S.toString(QueryTestKey.class, this);
         }
+
+        /** {@inheritDoc} */
+        @Override public int compareTo(Object o) {
+            return key - ((QueryTestKey)o).key;
+        }
     }
 
     /**
@@ -644,6 +1261,7 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
             return S.toString(QueryTestValue.class, this);
         }
     }
+
     /**
      *
      */
@@ -681,4 +1299,10 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
         }
     }
 
+    /**
+     *
+     */
+    protected enum ContinuousDeploy {
+        CLIENT, SERVER, ALL
+    }
 }


[13/21] ignite git commit: IGNITE-2156 .Net: Added ClientDisconnectedException to API. This closes #397.

Posted by sb...@apache.org.
IGNITE-2156 .Net: Added ClientDisconnectedException to API. This closes #397.


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

Branch: refs/heads/ignite-1232
Commit: acaeafb84ca46a402ccb75d59620d197a7c549fe
Parents: 8562b00
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Feb 17 16:11:36 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Feb 17 16:11:36 2016 +0300

----------------------------------------------------------------------
 .../platform/PlatformProcessorImpl.java         |  17 ++-
 .../callback/PlatformCallbackGateway.java       |  30 +++++
 .../callback/PlatformCallbackUtils.java         |  15 +++
 .../cpp/common/include/ignite/common/java.h     |   9 ++
 modules/platforms/cpp/common/src/java.cpp       |  16 ++-
 .../Apache.Ignite.Core.Tests.csproj             |   2 +
 .../Process/IgniteProcess.cs                    |  16 +++
 .../ProcessExtensions.cs                        |  78 +++++++++++++
 .../Apache.Ignite.Core.Tests/ReconnectTest.cs   |  96 ++++++++++++++++
 .../Apache.Ignite.Core.csproj                   |   1 +
 .../Apache.Ignite.Core/Cluster/ICluster.cs      |  14 +++
 .../Common/ClientDisconnectedException.cs       |  97 ++++++++++++++++
 .../Impl/Binary/BinaryUtils.cs                  |   2 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |   2 +-
 .../Impl/Compute/ComputeTaskHolder.cs           |   2 +-
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   | 110 +++++++++++--------
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  32 +++++-
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |   7 ++
 .../Impl/Unmanaged/UnmanagedCallbackHandlers.cs |   3 +
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        |  32 +++++-
 20 files changed, 524 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/acaeafb8/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
index d0e0a63..76967ff 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
@@ -27,7 +27,9 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.PlatformConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteComputeImpl;
-import org.apache.ignite.internal.binary.*;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.binary.BinaryRawReaderEx;
+import org.apache.ignite.internal.binary.BinaryRawWriterEx;
 import org.apache.ignite.internal.cluster.ClusterGroupAdapter;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
@@ -53,6 +55,7 @@ import org.apache.ignite.internal.processors.platform.utils.PlatformConfiguratio
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteFuture;
 import org.jetbrains.annotations.Nullable;
 
 import java.util.Collection;
@@ -379,6 +382,18 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
+        platformCtx.gateway().onClientDisconnected();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteInternalFuture<?> onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
+        platformCtx.gateway().onClientReconnected(clusterRestarted);
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
     @Override public void getIgniteConfiguration(long memPtr) {
         PlatformOutputStream stream = platformCtx.memory().get(memPtr).output();
         BinaryRawWriterEx writer = platformCtx.writer(stream);

http://git-wip-us.apache.org/repos/asf/ignite/blob/acaeafb8/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
index 47862a2..5093773 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
@@ -911,6 +911,36 @@ public class PlatformCallbackGateway {
     }
 
     /**
+     * Notifies platform about client disconnect.
+     */
+    public void onClientDisconnected() {
+        enter();
+
+        try {
+            PlatformCallbackUtils.onClientDisconnected(envPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Notifies platform about client reconnect.
+     *
+     * @param clusterRestarted Cluster restarted flag.
+     */
+    public void onClientReconnected(boolean clusterRestarted) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.onClientReconnected(envPtr, clusterRestarted);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
      * Kernal stop callback.
      */
     public void onStop() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/acaeafb8/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
index 3112e0f..f7d6586 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
@@ -481,6 +481,21 @@ public class PlatformCallbackUtils {
     static native long extensionCallbackInLongLongOutLong(long envPtr, int typ, long arg1, long arg2);
 
     /**
+     * Notifies platform about client disconnect.
+     *
+     * @param envPtr Environment pointer.
+     */
+    static native void onClientDisconnected(long envPtr);
+
+    /**
+     * Notifies platform about client reconnect.
+     *
+     * @param envPtr Environment pointer.
+     * @param clusterRestarted Cluster restarted flag.
+     */
+    static native void onClientReconnected(long envPtr, boolean clusterRestarted);
+
+    /**
      * Private constructor.
      */
     private PlatformCallbackUtils() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/acaeafb8/modules/platforms/cpp/common/include/ignite/common/java.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/common/java.h b/modules/platforms/cpp/common/include/ignite/common/java.h
index 8f5823e..ed47bc3 100644
--- a/modules/platforms/cpp/common/include/ignite/common/java.h
+++ b/modules/platforms/cpp/common/include/ignite/common/java.h
@@ -103,6 +103,9 @@ namespace ignite
             typedef long long(JNICALL *ExtensionCallbackInLongOutLongHandler)(void* target, int typ, long long arg1);
             typedef long long(JNICALL *ExtensionCallbackInLongLongOutLongHandler)(void* target, int typ, long long arg1, long long arg2);
 
+            typedef void(JNICALL *OnClientDisconnectedHandler)(void* target);
+            typedef void(JNICALL *OnClientReconnectedHandler)(void* target, unsigned char clusterRestarted);
+
             /**
              * JNI handlers holder.
              */
@@ -177,6 +180,9 @@ namespace ignite
 
                 ExtensionCallbackInLongOutLongHandler extensionCallbackInLongOutLong;
                 ExtensionCallbackInLongLongOutLongHandler extensionCallbackInLongLongOutLong;
+
+                OnClientDisconnectedHandler onClientDisconnected;
+                OnClientReconnectedHandler onClientReconnected;
             };
 
             /**
@@ -727,6 +733,9 @@ namespace ignite
 
             JNIEXPORT jlong JNICALL JniExtensionCallbackInLongOutLong(JNIEnv *env, jclass cls, jlong envPtr, jint typ, jlong arg1);
             JNIEXPORT jlong JNICALL JniExtensionCallbackInLongLongOutLong(JNIEnv *env, jclass cls, jlong envPtr, jint typ, jlong arg1, jlong arg2);
+
+            JNIEXPORT void JNICALL JniOnClientDisconnected(JNIEnv *env, jclass cls, jlong envPtr);
+            JNIEXPORT void JNICALL JniOnClientReconnected(JNIEnv *env, jclass cls, jlong envPtr, jboolean clusterRestarted);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/acaeafb8/modules/platforms/cpp/common/src/java.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/src/java.cpp b/modules/platforms/cpp/common/src/java.cpp
index d6f7ef0..8fc2293 100644
--- a/modules/platforms/cpp/common/src/java.cpp
+++ b/modules/platforms/cpp/common/src/java.cpp
@@ -352,6 +352,9 @@ namespace ignite
             JniMethod M_PLATFORM_CALLBACK_UTILS_EXTENSION_CALLBACK_IN_LONG_OUT_LONG = JniMethod("extensionCallbackInLongOutLong", "(JIJ)J", true);
             JniMethod M_PLATFORM_CALLBACK_UTILS_EXTENSION_CALLBACK_IN_LONG_LONG_OUT_LONG = JniMethod("extensionCallbackInLongLongOutLong", "(JIJJ)J", true);
 
+            JniMethod M_PLATFORM_CALLBACK_UTILS_ON_CLIENT_DISCONNECTED = JniMethod("onClientDisconnected", "(J)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_ON_CLIENT_RECONNECTED = JniMethod("onClientReconnected", "(JZ)V", true);
+
             const char* C_PLATFORM_UTILS = "org/apache/ignite/internal/processors/platform/utils/PlatformUtils";
             JniMethod M_PLATFORM_UTILS_REALLOC = JniMethod("reallocate", "(JI)V", true);
             JniMethod M_PLATFORM_UTILS_ERR_DATA = JniMethod("errorData", "(Ljava/lang/Throwable;)[B", true);
@@ -808,7 +811,7 @@ namespace ignite
 
             void RegisterNatives(JNIEnv* env) {
                 {
-					JNINativeMethod methods[52];
+					JNINativeMethod methods[54];
 
                     int idx = 0;
 
@@ -882,6 +885,9 @@ namespace ignite
                     AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_EXTENSION_CALLBACK_IN_LONG_OUT_LONG, reinterpret_cast<void*>(JniExtensionCallbackInLongOutLong));
                     AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_EXTENSION_CALLBACK_IN_LONG_LONG_OUT_LONG, reinterpret_cast<void*>(JniExtensionCallbackInLongLongOutLong));
 
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_ON_CLIENT_DISCONNECTED, reinterpret_cast<void*>(JniOnClientDisconnected));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_ON_CLIENT_RECONNECTED, reinterpret_cast<void*>(JniOnClientReconnected));
+
                     jint res = env->RegisterNatives(FindClass(env, C_PLATFORM_CALLBACK_UTILS), methods, idx);
 
                     if (res != JNI_OK)
@@ -2716,6 +2722,14 @@ namespace ignite
             JNIEXPORT jlong JNICALL JniExtensionCallbackInLongLongOutLong(JNIEnv *env, jclass cls, jlong envPtr, jint typ, jlong arg1, jlong arg2) {
                 IGNITE_SAFE_FUNC(env, envPtr, ExtensionCallbackInLongLongOutLongHandler, extensionCallbackInLongLongOutLong, typ, arg1, arg2);
             }
+            
+            JNIEXPORT void JNICALL JniOnClientDisconnected(JNIEnv *env, jclass cls, jlong envPtr) {
+                IGNITE_SAFE_PROC_NO_ARG(env, envPtr, OnClientDisconnectedHandler, onClientDisconnected);
+            }
+
+            JNIEXPORT void JNICALL JniOnClientReconnected(JNIEnv *env, jclass cls, jlong envPtr, jboolean clusterRestarted) {
+                IGNITE_SAFE_PROC(env, envPtr, OnClientReconnectedHandler, onClientReconnected, clusterRestarted);
+            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/acaeafb8/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index fb14ed5..6f0e630 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -129,6 +129,8 @@
     <Compile Include="MessagingTest.cs" />
     <Compile Include="BinaryConfigurationTest.cs" />
     <Compile Include="Binary\BinaryStructureTest.cs" />
+    <Compile Include="ProcessExtensions.cs" />
+    <Compile Include="ReconnectTest.cs" />
     <Compile Include="SerializationTest.cs" />
     <Compile Include="IgniteStartStopTest.cs" />
     <Compile Include="TestUtils.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/acaeafb8/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs
index 4853d93..85464e9 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs
@@ -225,6 +225,22 @@ namespace Apache.Ignite.Core.Tests.Process
         }
 
         /// <summary>
+        /// Suspends the process.
+        /// </summary>
+        public void Suspend()
+        {
+            _proc.Suspend();
+        }
+
+        /// <summary>
+        /// Resumes the process.
+        /// </summary>
+        public void Resume()
+        {
+            _proc.Resume();
+        }
+
+        /// <summary>
         /// Join process.
         /// </summary>
         /// <returns>Exit code.</returns>

http://git-wip-us.apache.org/repos/asf/ignite/blob/acaeafb8/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProcessExtensions.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProcessExtensions.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProcessExtensions.cs
new file mode 100644
index 0000000..b4c0a27
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProcessExtensions.cs
@@ -0,0 +1,78 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests
+{
+    using System;
+    using System.Diagnostics;
+    using System.Linq;
+    using System.Runtime.InteropServices;
+
+    /// <summary>
+    /// Process extensions.
+    /// </summary>
+    public static class ProcessExtensions
+    {
+        /** */
+        private const int ThreadAccessSuspendResume = 0x2;
+
+        /** */
+        [DllImport("kernel32.dll")]
+        private static extern IntPtr OpenThread(int dwDesiredAccess, bool bInheritHandle, uint dwThreadId);
+
+        /** */
+        [DllImport("kernel32.dll")]
+        private static extern uint SuspendThread(IntPtr hThread);
+
+        /** */
+        [DllImport("kernel32.dll")]
+        private static extern int ResumeThread(IntPtr hThread);
+
+        /// <summary>
+        /// Suspends the specified process.
+        /// </summary>
+        /// <param name="process">The process.</param>
+        public static void Suspend(this System.Diagnostics.Process process)
+        {
+            foreach (var thread in process.Threads.Cast<ProcessThread>())
+            {
+                var pOpenThread = OpenThread(ThreadAccessSuspendResume, false, (uint)thread.Id);
+
+                if (pOpenThread == IntPtr.Zero)
+                    break;
+
+                SuspendThread(pOpenThread);
+            }
+        }
+        /// <summary>
+        /// Resumes the specified process.
+        /// </summary>
+        /// <param name="process">The process.</param>
+        public static void Resume(this System.Diagnostics.Process process)
+        {
+            foreach (var thread in process.Threads.Cast<ProcessThread>())
+            {
+                var pOpenThread = OpenThread(ThreadAccessSuspendResume, false, (uint)thread.Id);
+
+                if (pOpenThread == IntPtr.Zero)
+                    break;
+
+                ResumeThread(pOpenThread);
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/acaeafb8/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs
new file mode 100644
index 0000000..5cb0a4f
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs
@@ -0,0 +1,96 @@
+/*
+ * 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.
+ */
+
+#pragma warning disable 618  // Deprecated SpringConfigUrl
+namespace Apache.Ignite.Core.Tests
+{
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Tests.Process;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Client reconnect tests.
+    /// </summary>
+    public class ReconnectTest
+    {
+        /// <summary>
+        /// Tests the disconnected exception.
+        /// </summary>
+        [Test]
+        public void TestDisconnectedException()
+        {
+            var cfg = new IgniteConfiguration
+            {
+                SpringConfigUrl = "config\\compute\\compute-grid1.xml",
+                JvmClasspath = TestUtils.CreateTestClasspath(),
+                JvmOptions = TestUtils.TestJavaOptions()
+            };
+
+            var proc = StartServerProcess(cfg);
+
+            Ignition.ClientMode = true;
+
+            using (var ignite = Ignition.Start(cfg))
+            {
+                Assert.IsTrue(ignite.GetCluster().ClientReconnectTask.IsCompleted);
+
+                var cache = ignite.GetCache<int, int>(null);
+
+                cache[1] = 1;
+
+                // Suspend external process to cause disconnect
+                proc.Suspend();
+
+                var ex = Assert.Throws<CacheException>(() => cache.Get(1));
+
+                var inner = (ClientDisconnectedException) ex.InnerException;
+
+                var clientReconnectTask = inner.ClientReconnectTask;
+
+                Assert.AreEqual(ignite.GetCluster().ClientReconnectTask, clientReconnectTask);
+
+                // Resume process to reconnect
+                proc.Resume();
+
+                clientReconnectTask.Wait();
+
+                Assert.AreEqual(1, cache[1]);
+            }
+        }
+
+        /// <summary>
+        /// Starts the server process.
+        /// </summary>
+        private static IgniteProcess StartServerProcess(IgniteConfiguration cfg)
+        {
+            return new IgniteProcess(
+                "-springConfigUrl=" + cfg.SpringConfigUrl, "-J-ea", "-J-Xcheck:jni", "-J-Xms512m", "-J-Xmx512m",
+                "-J-DIGNITE_QUIET=false");
+        }
+
+        /// <summary>
+        /// Fixture tear down.
+        /// </summary>
+        [TestFixtureTearDown]
+        public void FixtureTearDown()
+        {
+            IgniteProcess.KillAll();
+            Ignition.ClientMode = false;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/acaeafb8/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index d0ef352..661040b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -109,6 +109,7 @@
     <Compile Include="Common\IFactory.cs" />
     <Compile Include="Cache\Store\ICacheStoreSession.cs" />
     <Compile Include="Cache\Store\Package-Info.cs" />
+    <Compile Include="Common\ClientDisconnectedException.cs" />
     <Compile Include="Cluster\ClusterGroupEmptyException.cs" />
     <Compile Include="Cluster\ClusterTopologyException.cs" />
     <Compile Include="Cluster\ICluster.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/acaeafb8/modules/platforms/dotnet/Apache.Ignite.Core/Cluster/ICluster.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cluster/ICluster.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cluster/ICluster.cs
index e50970b..812a644 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cluster/ICluster.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cluster/ICluster.cs
@@ -20,6 +20,7 @@ namespace Apache.Ignite.Core.Cluster
     using System;
     using System.Collections.Generic;
     using System.Diagnostics.CodeAnalysis;
+    using System.Threading.Tasks;
     using Apache.Ignite.Core.Common;
 
     /// <summary>
@@ -75,5 +76,18 @@ namespace Apache.Ignite.Core.Cluster
         /// Resets local I/O, job, and task execution metrics.
         /// </summary>
         void ResetMetrics();
+
+        /// <summary>
+        /// Gets the reconnect task, which will transition to Completed state 
+        /// when local client node reconnects to the cluster. 
+        /// <para />
+        /// Result of the task indicates whether cluster has been restarted.
+        /// <para />
+        /// If local node is not in client mode or is not disconnected, returns completed task.
+        /// </summary>
+        /// <value>
+        /// The reconnect task.
+        /// </value>
+        Task<bool> ClientReconnectTask { get; } 
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/acaeafb8/modules/platforms/dotnet/Apache.Ignite.Core/Common/ClientDisconnectedException.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Common/ClientDisconnectedException.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Common/ClientDisconnectedException.cs
new file mode 100644
index 0000000..8843a0b
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Common/ClientDisconnectedException.cs
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Common
+{
+    using System;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Runtime.Serialization;
+    using System.Threading.Tasks;
+
+    /// <summary>
+    /// Indicates that client-mode local node has been disconnected from the cluster.
+    /// </summary>
+    [SuppressMessage("Microsoft.Usage", "CA2240:ImplementISerializableCorrectly", 
+        Justification = "No need to implement GetObjectData because there are no custom fields.")]
+    [Serializable]
+    public sealed class ClientDisconnectedException : IgniteException
+    {
+        /// <summary>
+        /// The client reconnect task.
+        /// </summary>
+        private readonly Task<bool> _clientReconnectTask;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ClientDisconnectedException"/> class.
+        /// </summary>
+        public ClientDisconnectedException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ClientDisconnectedException"/> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public ClientDisconnectedException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ClientDisconnectedException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public ClientDisconnectedException(string message, Exception cause) : base(message, cause)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ClientDisconnectedException" /> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        /// <param name="clientReconnectTask">The client reconnect task.</param>
+        public ClientDisconnectedException(string message, Exception cause, Task<bool> clientReconnectTask) : base(message, cause)
+        {
+            _clientReconnectTask = clientReconnectTask;
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ClientDisconnectedException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        private ClientDisconnectedException(SerializationInfo info, StreamingContext ctx) : base(info, ctx)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Gets the client reconnect task, if present.
+        /// </summary>
+        /// <value>
+        /// The client reconnect task, or null.
+        /// </value>
+        public Task<bool> ClientReconnectTask
+        {
+            get { return _clientReconnectTask; }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/acaeafb8/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
index 9066bd1..b73a6c4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
@@ -1658,7 +1658,7 @@ namespace Apache.Ignite.Core.Impl.Binary
 
             err = reader.ReadBoolean()
                 ? reader.ReadObject<object>()
-                : ExceptionUtils.GetException(reader.ReadString(), reader.ReadString());
+                : ExceptionUtils.GetException(reader.Marshaller.Ignite, reader.ReadString(), reader.ReadString());
 
             return null;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/acaeafb8/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs
index d1296ec..1296596 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs
@@ -1170,7 +1170,7 @@ namespace Apache.Ignite.Core.Impl.Cache
 
             var msg = Unmarshal<string>(inStream);
                 
-            return new CacheEntryProcessorException(ExceptionUtils.GetException(clsName, msg));
+            return new CacheEntryProcessorException(ExceptionUtils.GetException(_ignite, clsName, msg));
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/acaeafb8/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeTaskHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeTaskHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeTaskHolder.cs
index e992245..a7988c5 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeTaskHolder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeTaskHolder.cs
@@ -391,7 +391,7 @@ namespace Apache.Ignite.Core.Impl.Compute
             {
                 err = reader.ReadBoolean()
                     ? reader.ReadObject<BinaryObject>().Deserialize<Exception>()
-                    : ExceptionUtils.GetException(reader.ReadString(), reader.ReadString());
+                    : ExceptionUtils.GetException(_compute.Marshaller.Ignite, reader.ReadString(), reader.ReadString());
             }
             catch (Exception e)
             {

http://git-wip-us.apache.org/repos/asf/ignite/blob/acaeafb8/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
index 4d2e458..695f156 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
@@ -22,6 +22,7 @@ namespace Apache.Ignite.Core.Impl
     using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
     using System.Security;
+    using System.Text.RegularExpressions;
     using System.Threading;
     using Apache.Ignite.Core.Cache;
     using Apache.Ignite.Core.Cache.Store;
@@ -44,102 +45,119 @@ namespace Apache.Ignite.Core.Impl
 
         /** InteropCachePartialUpdateException. */
         private const string ClsCachePartialUpdateErr = "org.apache.ignite.internal.processors.platform.cache.PlatformCachePartialUpdateException";
-        
+
         /** Map with predefined exceptions. */
         private static readonly IDictionary<string, ExceptionFactoryDelegate> Exs = new Dictionary<string, ExceptionFactoryDelegate>();
 
         /** Exception factory delegate. */
-        private delegate Exception ExceptionFactoryDelegate(string msg);
-        
+        private delegate Exception ExceptionFactoryDelegate(IIgnite ignite, string msg, Exception innerEx);
+
+        /** Inner class regex. */
+        private static readonly Regex InnerClassRegex = new Regex(@"class ([^\s]+): (.*)", RegexOptions.Compiled);
+
         /// <summary>
         /// Static initializer.
         /// </summary>
-        [SuppressMessage("Microsoft.Performance", "CA1810:InitializeReferenceTypeStaticFieldsInline", 
+        [SuppressMessage("Microsoft.Performance", "CA1810:InitializeReferenceTypeStaticFieldsInline",
             Justification = "Readability")]
         static ExceptionUtils()
         {
             // Common Java exceptions mapped to common .Net exceptions.
-            Exs["java.lang.IllegalArgumentException"] = m => new ArgumentException(m);
-            Exs["java.lang.IllegalStateException"] = m => new InvalidOperationException(m);
-            Exs["java.lang.UnsupportedOperationException"] = m => new NotImplementedException(m);
-            Exs["java.lang.InterruptedException"] = m => new ThreadInterruptedException(m);
-            
+            Exs["java.lang.IllegalArgumentException"] = (i, m, e) => new ArgumentException(m, e);
+            Exs["java.lang.IllegalStateException"] = (i, m, e) => new InvalidOperationException(m, e);
+            Exs["java.lang.UnsupportedOperationException"] = (i, m, e) => new NotImplementedException(m, e);
+            Exs["java.lang.InterruptedException"] = (i, m, e) => new ThreadInterruptedException(m, e);
+
             // Generic Ignite exceptions.
-            Exs["org.apache.ignite.IgniteException"] = m => new IgniteException(m);
-            Exs["org.apache.ignite.IgniteCheckedException"] = m => new IgniteException(m);
+            Exs["org.apache.ignite.IgniteException"] = (i, m, e) => new IgniteException(m, e);
+            Exs["org.apache.ignite.IgniteCheckedException"] = (i, m, e) => new IgniteException(m, e);
+            Exs["org.apache.ignite.IgniteClientDisconnectedException"] = (i, m, e) => new ClientDisconnectedException(m, e, i.GetCluster().ClientReconnectTask);
+            Exs["org.apache.ignite.internal.IgniteClientDisconnectedCheckedException"] = (i, m, e) => new ClientDisconnectedException(m, e, i.GetCluster().ClientReconnectTask);
 
             // Cluster exceptions.
-            Exs["org.apache.ignite.cluster.ClusterGroupEmptyException"] = m => new ClusterGroupEmptyException(m);
-            Exs["org.apache.ignite.cluster.ClusterTopologyException"] = m => new ClusterTopologyException(m);
+            Exs["org.apache.ignite.cluster.ClusterGroupEmptyException"] = (i, m, e) => new ClusterGroupEmptyException(m, e);
+            Exs["org.apache.ignite.cluster.ClusterTopologyException"] = (i, m, e) => new ClusterTopologyException(m, e);
 
             // Compute exceptions.
-            Exs["org.apache.ignite.compute.ComputeExecutionRejectedException"] = m => new ComputeExecutionRejectedException(m);
-            Exs["org.apache.ignite.compute.ComputeJobFailoverException"] = m => new ComputeJobFailoverException(m);
-            Exs["org.apache.ignite.compute.ComputeTaskCancelledException"] = m => new ComputeTaskCancelledException(m);
-            Exs["org.apache.ignite.compute.ComputeTaskTimeoutException"] = m => new ComputeTaskTimeoutException(m);
-            Exs["org.apache.ignite.compute.ComputeUserUndeclaredException"] = m => new ComputeUserUndeclaredException(m);
+            Exs["org.apache.ignite.compute.ComputeExecutionRejectedException"] = (i, m, e) => new ComputeExecutionRejectedException(m, e);
+            Exs["org.apache.ignite.compute.ComputeJobFailoverException"] = (i, m, e) => new ComputeJobFailoverException(m, e);
+            Exs["org.apache.ignite.compute.ComputeTaskCancelledException"] = (i, m, e) => new ComputeTaskCancelledException(m, e);
+            Exs["org.apache.ignite.compute.ComputeTaskTimeoutException"] = (i, m, e) => new ComputeTaskTimeoutException(m, e);
+            Exs["org.apache.ignite.compute.ComputeUserUndeclaredException"] = (i, m, e) => new ComputeUserUndeclaredException(m, e);
 
             // Cache exceptions.
-            Exs["javax.cache.CacheException"] = m => new CacheException(m);
-            Exs["javax.cache.integration.CacheLoaderException"] = m => new CacheStoreException(m);
-            Exs["javax.cache.integration.CacheWriterException"] = m => new CacheStoreException(m);
-            Exs["javax.cache.processor.EntryProcessorException"] = m => new CacheEntryProcessorException(m);
-            Exs["org.apache.ignite.cache.CacheAtomicUpdateTimeoutException"] = m => new CacheAtomicUpdateTimeoutException(m);
-            
+            Exs["javax.cache.CacheException"] = (i, m, e) => new CacheException(m, e);
+            Exs["javax.cache.integration.CacheLoaderException"] = (i, m, e) => new CacheStoreException(m, e);
+            Exs["javax.cache.integration.CacheWriterException"] = (i, m, e) => new CacheStoreException(m, e);
+            Exs["javax.cache.processor.EntryProcessorException"] = (i, m, e) => new CacheEntryProcessorException(m, e);
+            Exs["org.apache.ignite.cache.CacheAtomicUpdateTimeoutException"] = (i, m, e) => new CacheAtomicUpdateTimeoutException(m, e);
+
             // Transaction exceptions.
-            Exs["org.apache.ignite.transactions.TransactionOptimisticException"] = m => new TransactionOptimisticException(m);
-            Exs["org.apache.ignite.transactions.TransactionTimeoutException"] = m => new TransactionTimeoutException(m);
-            Exs["org.apache.ignite.transactions.TransactionRollbackException"] = m => new TransactionRollbackException(m);
-            Exs["org.apache.ignite.transactions.TransactionHeuristicException"] = m => new TransactionHeuristicException(m);
+            Exs["org.apache.ignite.transactions.TransactionOptimisticException"] = (i, m, e) => new TransactionOptimisticException(m, e);
+            Exs["org.apache.ignite.transactions.TransactionTimeoutException"] = (i, m, e) => new TransactionTimeoutException(m, e);
+            Exs["org.apache.ignite.transactions.TransactionRollbackException"] = (i, m, e) => new TransactionRollbackException(m, e);
+            Exs["org.apache.ignite.transactions.TransactionHeuristicException"] = (i, m, e) => new TransactionHeuristicException(m, e);
 
             // Security exceptions.
-            Exs["org.apache.ignite.IgniteAuthenticationException"] = m => new SecurityException(m);
-            Exs["org.apache.ignite.plugin.security.GridSecurityException"] = m => new SecurityException(m);
+            Exs["org.apache.ignite.IgniteAuthenticationException"] = (i, m, e) => new SecurityException(m, e);
+            Exs["org.apache.ignite.plugin.security.GridSecurityException"] = (i, m, e) => new SecurityException(m, e);
 
             // Future exceptions
-            Exs["org.apache.ignite.lang.IgniteFutureCancelledException"] = m => new IgniteFutureCancelledException(m);
-            Exs["org.apache.ignite.internal.IgniteFutureCancelledCheckedException"] = m => new IgniteFutureCancelledException(m);
+            Exs["org.apache.ignite.lang.IgniteFutureCancelledException"] = (i, m, e) => new IgniteFutureCancelledException(m, e);
+            Exs["org.apache.ignite.internal.IgniteFutureCancelledCheckedException"] = (i, m, e) => new IgniteFutureCancelledException(m, e);
         }
 
         /// <summary>
         /// Creates exception according to native code class and message.
         /// </summary>
+        /// <param name="ignite">The ignite.</param>
         /// <param name="clsName">Exception class name.</param>
         /// <param name="msg">Exception message.</param>
         /// <param name="reader">Error data reader.</param>
-        public static Exception GetException(string clsName, string msg, BinaryReader reader = null)
+        /// <returns>Exception.</returns>
+        public static Exception GetException(IIgnite ignite, string clsName, string msg, BinaryReader reader = null)
         {
             ExceptionFactoryDelegate ctor;
 
             if (Exs.TryGetValue(clsName, out ctor))
-                return ctor(msg);
+            {
+                var match = InnerClassRegex.Match(msg);
 
-            if (ClsNoClsDefFoundErr.Equals(clsName))
+                ExceptionFactoryDelegate innerCtor;
+
+                if (match.Success && Exs.TryGetValue(match.Groups[1].Value, out innerCtor))
+                    return ctor(ignite, msg, innerCtor(ignite, match.Groups[2].Value, null));
+
+                return ctor(ignite, msg, null);
+            }
+
+            if (ClsNoClsDefFoundErr.Equals(clsName, StringComparison.OrdinalIgnoreCase))
                 return new IgniteException("Java class is not found (did you set IGNITE_HOME environment " +
                     "variable?): " + msg);
 
-            if (ClsNoSuchMthdErr.Equals(clsName))
+            if (ClsNoSuchMthdErr.Equals(clsName, StringComparison.OrdinalIgnoreCase))
                 return new IgniteException("Java class method is not found (did you set IGNITE_HOME environment " +
                     "variable?): " + msg);
 
-            if (ClsCachePartialUpdateErr.Equals(clsName))
-                return ProcessCachePartialUpdateException(msg, reader);
-            
+            if (ClsCachePartialUpdateErr.Equals(clsName, StringComparison.OrdinalIgnoreCase))
+                return ProcessCachePartialUpdateException(ignite, msg, reader);
+
             return new IgniteException("Java exception occurred [class=" + clsName + ", message=" + msg + ']');
         }
 
         /// <summary>
         /// Process cache partial update exception.
         /// </summary>
+        /// <param name="ignite">The ignite.</param>
         /// <param name="msg">Message.</param>
         /// <param name="reader">Reader.</param>
-        /// <returns></returns>
+        /// <returns>CachePartialUpdateException.</returns>
         [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes")]
-        private static Exception ProcessCachePartialUpdateException(string msg, BinaryReader reader)
+        private static Exception ProcessCachePartialUpdateException(IIgnite ignite, string msg, BinaryReader reader)
         {
             if (reader == null)
                 return new CachePartialUpdateException(msg, new IgniteException("Failed keys are not available."));
-            
+
             bool dataExists = reader.ReadBoolean();
 
             Debug.Assert(dataExists);
@@ -160,12 +178,12 @@ namespace Apache.Ignite.Core.Impl
                     return new CachePartialUpdateException(msg, e);
                 }
             }
-            
+
             // Was not able to write keys.
             string innerErrCls = reader.ReadString();
             string innerErrMsg = reader.ReadString();
 
-            Exception innerErr = GetException(innerErrCls, innerErrMsg);
+            Exception innerErr = GetException(ignite, innerErrCls, innerErrMsg);
 
             return new CachePartialUpdateException(msg, innerErr);
         }
@@ -179,7 +197,7 @@ namespace Apache.Ignite.Core.Impl
         public static Exception GetJvmInitializeException(string clsName, string msg)
         {
             if (clsName != null)
-                return new IgniteException("Failed to initialize JVM.", GetException(clsName, msg));
+                return new IgniteException("Failed to initialize JVM.", GetException(null, clsName, msg));
 
             if (msg != null)
                 return new IgniteException("Failed to initialize JVM: " + msg);
@@ -194,7 +212,7 @@ namespace Apache.Ignite.Core.Impl
         /// <returns>List.</returns>
         private static List<object> ReadNullableList(BinaryReader reader)
         {
-            if (!reader.ReadBoolean()) 
+            if (!reader.ReadBoolean())
                 return null;
 
             var size = reader.ReadInt();

http://git-wip-us.apache.org/repos/asf/ignite/blob/acaeafb8/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
index be21d7f..0271fa2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
@@ -23,6 +23,7 @@ namespace Apache.Ignite.Core.Impl
     using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
     using System.Linq;
+    using System.Threading.Tasks;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Cache;
     using Apache.Ignite.Core.Cache.Configuration;
@@ -86,10 +87,13 @@ namespace Apache.Ignite.Core.Impl
         private readonly UnmanagedCallbacks _cbs;
 
         /** Node info cache. */
-
         private readonly ConcurrentDictionary<Guid, ClusterNodeImpl> _nodes =
             new ConcurrentDictionary<Guid, ClusterNodeImpl>();
 
+        /** Client reconnect task completion source. */
+        private volatile TaskCompletionSource<bool> _clientReconnectTaskCompletionSource = 
+            new TaskCompletionSource<bool>();
+
         /// <summary>
         /// Constructor.
         /// </summary>
@@ -128,6 +132,9 @@ namespace Apache.Ignite.Core.Impl
             // Grid is not completely started here, can't initialize interop transactions right away.
             _transactions = new Lazy<TransactionsImpl>(
                     () => new TransactionsImpl(UU.ProcessorTransactions(proc), marsh, GetLocalNode().Id));
+
+            // Set reconnected task to completed state for convenience.
+            _clientReconnectTaskCompletionSource.SetResult(false);
         }
 
         /// <summary>
@@ -429,6 +436,12 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /** <inheritdoc /> */
+        public Task<bool> ClientReconnectTask
+        {
+            get { return _clientReconnectTaskCompletionSource.Task; }
+        }
+
+        /** <inheritdoc /> */
         public IDataStreamer<TK, TV> GetDataStreamer<TK, TV>(string cacheName)
         {
             return new DataStreamerImpl<TK, TV>(UU.ProcessorDataStreamer(_proc, cacheName, false),
@@ -630,5 +643,22 @@ namespace Apache.Ignite.Core.Impl
         {
             get { return _proc; }
         }
+
+        /// <summary>
+        /// Called when local client node has been disconnected from the cluster.
+        /// </summary>
+        public void OnClientDisconnected()
+        {
+            _clientReconnectTaskCompletionSource = new TaskCompletionSource<bool>();
+        }
+
+        /// <summary>
+        /// Called when local client node has been reconnected to the cluster.
+        /// </summary>
+        /// <param name="clusterRestarted">Cluster restarted flag.</param>
+        public void OnClientReconnected(bool clusterRestarted)
+        {
+            _clientReconnectTaskCompletionSource.TrySetResult(clusterRestarted);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/acaeafb8/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
index a303783..0aa55fb 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
@@ -20,6 +20,7 @@ namespace Apache.Ignite.Core.Impl
     using System;
     using System.Collections.Generic;
     using System.Diagnostics.CodeAnalysis;
+    using System.Threading.Tasks;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Cache;
     using Apache.Ignite.Core.Cache.Configuration;
@@ -285,6 +286,12 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /** <inheritdoc /> */
+        public Task<bool> ClientReconnectTask
+        {
+            get { return _ignite.GetCluster().ClientReconnectTask; }
+        }
+
+        /** <inheritdoc /> */
         public IDataStreamer<TK, TV> GetDataStreamer<TK, TV>(string cacheName)
         {
             return _ignite.GetDataStreamer<TK, TV>(cacheName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/acaeafb8/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs
index 8147e9d..fb52033 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs
@@ -95,5 +95,8 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         internal void* extensionCbInLongOutLong;
         internal void* extensionCbInLongLongOutLong;
+
+        internal void* onClientDisconnected;
+        internal void* ocClientReconnected;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/acaeafb8/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
index 7778484..8d810e3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
@@ -162,6 +162,9 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
         private delegate long ExtensionCallbackInLongOutLongDelegate(void* target, int typ, long arg1);
         private delegate long ExtensionCallbackInLongLongOutLongDelegate(void* target, int typ, long arg1, long arg2);
 
+        private delegate void OnClientDisconnectedDelegate(void* target);
+        private delegate void OnClientReconnectedDelegate(void* target, bool clusterRestarted);
+
         /// <summary>
         /// constructor.
         /// </summary>
@@ -241,7 +244,10 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
                 error = CreateFunctionPointer((ErrorCallbackDelegate)Error),
                 
                 extensionCbInLongOutLong = CreateFunctionPointer((ExtensionCallbackInLongOutLongDelegate)ExtensionCallbackInLongOutLong),
-                extensionCbInLongLongOutLong = CreateFunctionPointer((ExtensionCallbackInLongLongOutLongDelegate)ExtensionCallbackInLongLongOutLong)
+                extensionCbInLongLongOutLong = CreateFunctionPointer((ExtensionCallbackInLongLongOutLongDelegate)ExtensionCallbackInLongLongOutLong),
+
+                onClientDisconnected = CreateFunctionPointer((OnClientDisconnectedDelegate)OnClientDisconnected),
+                ocClientReconnected = CreateFunctionPointer((OnClientReconnectedDelegate)OnClientReconnected),
             };
 
             _cbsPtr = Marshal.AllocHGlobal(UU.HandlersSize());
@@ -728,7 +734,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
                     string errCls = reader.ReadString();
                     string errMsg = reader.ReadString();
 
-                    Exception err = ExceptionUtils.GetException(errCls, errMsg, reader);
+                    Exception err = ExceptionUtils.GetException(_ignite, errCls, errMsg, reader);
 
                     ProcessFuture(futPtr, fut => { fut.OnError(err); });
                 }
@@ -1043,10 +1049,10 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
                         // Stream disposal intentionally omitted: IGNITE-1598
                         var stream = new PlatformRawMemory(errData, errDataLen).GetStream();
 
-                        throw ExceptionUtils.GetException(errCls, errMsg, _ignite.Marshaller.StartUnmarshal(stream));
+                        throw ExceptionUtils.GetException(_ignite, errCls, errMsg, _ignite.Marshaller.StartUnmarshal(stream));
                     }
 
-                    throw ExceptionUtils.GetException(errCls, errMsg);
+                    throw ExceptionUtils.GetException(_ignite, errCls, errMsg);
 
                 case ErrJvmInit:
                     throw ExceptionUtils.GetJvmInitializeException(errCls, errMsg);
@@ -1059,8 +1065,24 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
             }
         }
 
+        private void OnClientDisconnected(void* target)
+        {
+            SafeCall(() =>
+            {
+                _ignite.OnClientDisconnected();
+            });
+        }
+
+        private void OnClientReconnected(void* target, bool clusterRestarted)
+        {
+            SafeCall(() =>
+            {
+                _ignite.OnClientReconnected(clusterRestarted);
+            });
+        }
+
         #endregion
-        
+
         #region HELPERS
 
         [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes")]


[06/21] ignite git commit: Fixed CacheEvictableEntryImpl.equals()

Posted by sb...@apache.org.
Fixed CacheEvictableEntryImpl.equals()


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

Branch: refs/heads/ignite-1232
Commit: d2767359a339035df435e6484d48746db6333b83
Parents: 7be1aab
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Tue Feb 16 17:38:44 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Tue Feb 16 17:38:44 2016 -0800

----------------------------------------------------------------------
 .../GridCacheEvictableEntryEqualsSelfTest.java  | 85 ++++++++++++++++++++
 .../IgniteCacheEvictionSelfTestSuite.java       |  4 +-
 2 files changed, 88 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d2767359/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictableEntryEqualsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictableEntryEqualsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictableEntryEqualsSelfTest.java
new file mode 100644
index 0000000..2b40365
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictableEntryEqualsSelfTest.java
@@ -0,0 +1,85 @@
+/*
+ * 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.eviction;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.eviction.EvictableEntry;
+import org.apache.ignite.cache.eviction.EvictionPolicy;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Test for EvictableEntry.equals().
+ */
+public class GridCacheEvictableEntryEqualsSelfTest extends GridCommonAbstractTest {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEquals() throws Exception {
+        try (Ignite ignite = startGrid()) {
+            CacheConfiguration<TestKey, String> cfg = new CacheConfiguration<>("test");
+
+            cfg.setEvictionPolicy(new TestEvictionPolicy());
+
+            IgniteCache<TestKey, String> cache = ignite.createCache(cfg);
+
+            for (int i = 0; i < 10; i++)
+                cache.put(new TestKey(0), "val" + i);
+        }
+    }
+
+    private static class TestEvictionPolicy implements EvictionPolicy<TestKey, String>, Serializable {
+        private final Collection<EvictableEntry> entries = new ArrayList<>();
+
+        @Override public synchronized void onEntryAccessed(boolean rmv, EvictableEntry<TestKey, String> e) {
+            for (EvictableEntry e0 : entries)
+                assertTrue(e0.equals(e));
+
+            entries.add(e);
+        }
+    }
+
+    private static class TestKey {
+        private final int key;
+
+        public TestKey(int key) {
+            this.key = key;
+        }
+
+        @Override public boolean equals(Object other) {
+            if (this == other)
+                return true;
+
+            if (other == null || getClass() != other.getClass())
+                return false;
+
+            TestKey testKey = (TestKey)other;
+
+            return key == testKey.key;
+
+        }
+
+        @Override public int hashCode() {
+            return key;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d2767359/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
index 3d8bbba..1ad63ee 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.cache.eviction.GridCacheConcurrentE
 import org.apache.ignite.internal.processors.cache.eviction.GridCacheDistributedEvictionsSelfTest;
 import org.apache.ignite.internal.processors.cache.eviction.GridCacheEmptyEntriesLocalSelfTest;
 import org.apache.ignite.internal.processors.cache.eviction.GridCacheEmptyEntriesPartitionedSelfTest;
+import org.apache.ignite.internal.processors.cache.eviction.GridCacheEvictableEntryEqualsSelfTest;
 import org.apache.ignite.internal.processors.cache.eviction.GridCacheEvictionFilterSelfTest;
 import org.apache.ignite.internal.processors.cache.eviction.GridCacheEvictionLockUnlockSelfTest;
 import org.apache.ignite.internal.processors.cache.eviction.GridCacheEvictionTouchSelfTest;
@@ -68,7 +69,8 @@ public class IgniteCacheEvictionSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(GridCacheEmptyEntriesLocalSelfTest.class));
         suite.addTest(new TestSuite(GridCacheMemoryModeSelfTest.class));
         suite.addTest(new TestSuite(GridCacheSynchronousEvictionsFailoverSelfTest.class));
+        suite.addTest(new TestSuite(GridCacheEvictableEntryEqualsSelfTest.class));
 
         return suite;
     }
-}
\ No newline at end of file
+}


[03/21] ignite git commit: Fixed "IGNITE-2515 Make 'updateCntr' available through CacheContinuousQueryEvent public API"

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/cb35e1d7/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index c67a8cf..457ab9b 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -72,6 +72,10 @@ import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalAtomicQ
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.query.GridCacheSwapScanQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryCounterPartitionedAtomicTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryCounterPartitionedTxTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryCounterReplicatedAtomicTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryCounterReplicatedTxTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousBatchAckTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousBatchForceServerModeAckTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverAtomicPrimaryWriteOrderOffheapTieredTest;
@@ -213,6 +217,10 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(CacheContinuousQueryFailoverAtomicReplicatedSelfTest.class);
         suite.addTestSuite(CacheContinuousQueryFailoverTxSelfTest.class);
         suite.addTestSuite(CacheContinuousQueryFailoverTxReplicatedSelfTest.class);
+        suite.addTestSuite(CacheContinuousQueryCounterPartitionedAtomicTest.class);
+        suite.addTestSuite(CacheContinuousQueryCounterPartitionedTxTest.class);
+        suite.addTestSuite(CacheContinuousQueryCounterReplicatedAtomicTest.class);
+        suite.addTestSuite(CacheContinuousQueryCounterReplicatedTxTest.class);
         suite.addTestSuite(CacheContinuousQueryFailoverAtomicPrimaryWriteOrderOffheapTieredTest.class);
         suite.addTestSuite(CacheContinuousQueryFailoverTxOffheapTieredTest.class);
         suite.addTestSuite(CacheContinuousQueryRandomOperationsTest.class);


[08/21] ignite git commit: Fixed "IGNITE-2466 OutOfMemory when ONHEAP_TIERED mode is used".

Posted by sb...@apache.org.
Fixed "IGNITE-2466 OutOfMemory when ONHEAP_TIERED mode is used".


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

Branch: refs/heads/ignite-1232
Commit: 933d314e246a64d21b92ca51194be6e1429a46d2
Parents: d276735
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Feb 17 11:04:08 2016 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed Feb 17 11:04:08 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/GridJobCancelRequest.java       |  5 +++++
 .../ignite/internal/GridJobExecuteRequest.java      |  5 +++++
 .../ignite/internal/GridJobExecuteResponse.java     |  5 +++++
 .../ignite/internal/GridJobSiblingsRequest.java     |  5 +++++
 .../ignite/internal/GridJobSiblingsResponse.java    |  5 +++++
 .../ignite/internal/GridTaskCancelRequest.java      |  5 +++++
 .../ignite/internal/GridTaskSessionRequest.java     |  5 +++++
 .../internal/binary/BinaryEnumObjectImpl.java       |  5 +++++
 .../ignite/internal/binary/BinaryObjectImpl.java    |  5 +++++
 .../internal/binary/BinaryObjectOffheapImpl.java    |  5 +++++
 .../managers/checkpoint/GridCheckpointRequest.java  |  5 +++++
 .../managers/communication/GridIoMessage.java       |  5 +++++
 .../managers/communication/GridIoUserMessage.java   |  5 +++++
 .../managers/deployment/GridDeploymentInfoBean.java |  5 +++++
 .../managers/deployment/GridDeploymentRequest.java  |  5 +++++
 .../managers/deployment/GridDeploymentResponse.java |  5 +++++
 .../eventstorage/GridEventStorageMessage.java       |  5 +++++
 .../affinity/AffinityTopologyVersion.java           |  5 +++++
 .../processors/cache/CacheEntryInfoCollection.java  |  5 +++++
 .../cache/CacheEntryPredicateAdapter.java           |  5 +++++
 .../cache/CacheEntrySerializablePredicate.java      |  5 +++++
 .../processors/cache/CacheEvictionEntry.java        |  5 +++++
 .../processors/cache/CacheInvokeDirectResult.java   |  5 +++++
 .../processors/cache/CacheObjectByteArrayImpl.java  |  5 +++++
 .../internal/processors/cache/CacheObjectImpl.java  |  5 +++++
 .../processors/cache/GridCacheEntryInfo.java        |  5 +++++
 .../internal/processors/cache/GridCacheMessage.java |  5 +++++
 .../internal/processors/cache/GridCacheReturn.java  |  5 +++++
 .../processors/cache/KeyCacheObjectImpl.java        |  5 +++++
 .../distributed/dht/atomic/GridDhtAtomicCache.java  |  5 ++++-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java      | 14 ++++++++------
 .../dht/atomic/GridNearAtomicUpdateRequest.java     | 16 ++++++++++++++++
 .../dht/preloader/GridDhtPartitionExchangeId.java   |  5 +++++
 .../cache/distributed/near/CacheVersionedValue.java |  5 +++++
 .../processors/cache/query/GridCacheSqlQuery.java   |  5 +++++
 .../query/continuous/CacheContinuousQueryEntry.java |  5 +++++
 .../cache/transactions/IgniteTxEntry.java           |  5 +++++
 .../processors/cache/transactions/IgniteTxKey.java  |  5 +++++
 .../cache/transactions/TxEntryValueHolder.java      |  5 +++++
 .../processors/cache/version/GridCacheVersion.java  |  5 +++++
 .../clock/GridClockDeltaSnapshotMessage.java        |  5 +++++
 .../processors/clock/GridClockDeltaVersion.java     |  5 +++++
 .../continuous/GridContinuousMessage.java           |  5 +++++
 .../processors/datastreamer/DataStreamerEntry.java  |  5 +++++
 .../datastreamer/DataStreamerRequest.java           |  5 +++++
 .../datastreamer/DataStreamerResponse.java          |  5 +++++
 .../internal/processors/igfs/IgfsAckMessage.java    |  5 +++++
 .../internal/processors/igfs/IgfsBlockKey.java      |  5 +++++
 .../internal/processors/igfs/IgfsBlocksMessage.java |  5 +++++
 .../internal/processors/igfs/IgfsDeleteMessage.java |  5 +++++
 .../processors/igfs/IgfsFileAffinityRange.java      |  5 +++++
 .../processors/igfs/IgfsFragmentizerRequest.java    |  5 +++++
 .../processors/igfs/IgfsFragmentizerResponse.java   |  5 +++++
 .../internal/processors/igfs/IgfsSyncMessage.java   |  5 +++++
 .../h2/twostep/messages/GridQueryCancelRequest.java |  5 +++++
 .../h2/twostep/messages/GridQueryFailResponse.java  |  5 +++++
 .../twostep/messages/GridQueryNextPageRequest.java  |  5 +++++
 .../twostep/messages/GridQueryNextPageResponse.java |  5 +++++
 .../query/h2/twostep/messages/GridQueryRequest.java |  5 +++++
 .../rest/handlers/task/GridTaskResultRequest.java   |  5 +++++
 .../rest/handlers/task/GridTaskResultResponse.java  |  5 +++++
 .../ignite/internal/util/GridByteArrayList.java     |  5 +++++
 .../apache/ignite/internal/util/GridLongList.java   |  5 +++++
 .../ignite/internal/util/GridMessageCollection.java |  5 +++++
 .../ignite/internal/util/UUIDCollectionMessage.java |  5 +++++
 .../internal/util/nio/GridNioFinishedFuture.java    |  5 +++++
 .../ignite/internal/util/nio/GridNioFuture.java     |  7 ++++++-
 .../ignite/internal/util/nio/GridNioFutureImpl.java |  5 +++++
 .../util/nio/GridNioRecoveryDescriptor.java         |  2 ++
 .../ignite/internal/util/nio/GridNioServer.java     |  7 +++++++
 .../plugin/extensions/communication/Message.java    |  5 +++++
 .../collision/jobstealing/JobStealingRequest.java   |  5 +++++
 .../spi/communication/tcp/TcpCommunicationSpi.java  | 15 +++++++++++++++
 .../internal/GridAffinityNoCacheSelfTest.java       |  5 +++++
 .../GridCommunicationSendMessageSelfTest.java       |  5 +++++
 .../communication/GridIoManagerSelfTest.java        |  5 +++++
 .../CacheContinuousQueryCounterAbstractTest.java    | 11 +++++------
 .../loadtests/communication/GridTestMessage.java    |  5 +++++
 .../ignite/spi/communication/GridTestMessage.java   |  5 +++++
 .../query/h2/twostep/msg/GridH2ValueMessage.java    |  5 +++++
 .../query/h2/GridIndexingSpiAbstractSelfTest.java   |  5 +++++
 81 files changed, 428 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/GridJobCancelRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobCancelRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobCancelRequest.java
index 6222e8b..c9ce023 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobCancelRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobCancelRequest.java
@@ -111,6 +111,11 @@ public class GridJobCancelRequest implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteRequest.java
index 471c485..28b4094 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteRequest.java
@@ -422,6 +422,11 @@ public class GridJobExecuteRequest implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteResponse.java
index 6659b52..bfbd859 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteResponse.java
@@ -207,6 +207,11 @@ public class GridJobExecuteResponse implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsRequest.java
index 3554824..6fe6237 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsRequest.java
@@ -85,6 +85,11 @@ public class GridJobSiblingsRequest implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsResponse.java
index 925c2de..aa81cc5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsResponse.java
@@ -78,6 +78,11 @@ public class GridJobSiblingsResponse implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/GridTaskCancelRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridTaskCancelRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/GridTaskCancelRequest.java
index 39f562f..41ea281 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridTaskCancelRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridTaskCancelRequest.java
@@ -62,6 +62,11 @@ public class GridTaskCancelRequest implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionRequest.java
index 02d17db..15baac0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionRequest.java
@@ -99,6 +99,11 @@ public class GridTaskSessionRequest implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
index ab76b6e..33c62e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
@@ -231,6 +231,11 @@ public class BinaryEnumObjectImpl implements BinaryObjectEx, Externalizable, Cac
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public byte directType() {
         return 119;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/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 44c7a08..173bb6e 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
@@ -427,6 +427,11 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         if (detachAllowed) {
             int len = length();

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
index 07ab4d3..27d3012 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
@@ -392,6 +392,11 @@ public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Exter
         throw new UnsupportedOperationException(); // To make sure it is not marshalled.
     }
 
+    /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
     /**
      * @return Deserialized value.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointRequest.java
index 86d2d6c..95bf05b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointRequest.java
@@ -85,6 +85,11 @@ public class GridCheckpointRequest implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
index cb19ba0..b28ced2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
@@ -169,6 +169,11 @@ public class GridIoMessage implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        msg.onAckReceived();
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean equals(Object obj) {
         throw new AssertionError();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoUserMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoUserMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoUserMessage.java
index 50c34a8..9d03d3e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoUserMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoUserMessage.java
@@ -208,6 +208,11 @@ public class GridIoUserMessage implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfoBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfoBean.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfoBean.java
index 203cf7f..00ca67c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfoBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfoBean.java
@@ -133,6 +133,11 @@ public class GridDeploymentInfoBean implements Message, GridDeploymentInfo, Exte
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public int hashCode() {
         return clsLdrId.hashCode();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentRequest.java
index 5ebf7b6..72873b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentRequest.java
@@ -160,6 +160,11 @@ public class GridDeploymentRequest implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentResponse.java
index a4ac280..7a81f5a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentResponse.java
@@ -107,6 +107,11 @@ public class GridDeploymentResponse implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageMessage.java
index 44ede9a..8b06831 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageMessage.java
@@ -267,6 +267,11 @@ public class GridEventStorageMessage implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
index a43557f..80f0078 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityTopologyVersion.java
@@ -97,6 +97,11 @@ public class AffinityTopologyVersion implements Comparable<AffinityTopologyVersi
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean equals(Object o) {
         if (this == o)
             return true;

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInfoCollection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInfoCollection.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInfoCollection.java
index 55bc2a7..93bebec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInfoCollection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInfoCollection.java
@@ -59,6 +59,11 @@ public class CacheEntryInfoCollection implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateAdapter.java
index 10e4787..292ff18 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateAdapter.java
@@ -94,4 +94,9 @@ public abstract class CacheEntryPredicateAdapter implements CacheEntryPredicate
             throw new IgniteException(e);
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntrySerializablePredicate.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntrySerializablePredicate.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntrySerializablePredicate.java
index 20cc005..e44ef2b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntrySerializablePredicate.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntrySerializablePredicate.java
@@ -63,6 +63,11 @@ public class CacheEntrySerializablePredicate implements CacheEntryPredicate {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public void entryLocked(boolean locked) {
         assert p != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictionEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictionEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictionEntry.java
index 7bfdcb1..dae103a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictionEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictionEntry.java
@@ -105,6 +105,11 @@ public class CacheEvictionEntry implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeDirectResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeDirectResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeDirectResult.java
index fefa422..1e6d04b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeDirectResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeDirectResult.java
@@ -127,6 +127,11 @@ public class CacheInvokeDirectResult implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public byte directType() {
         return 93;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java
index e961d84..0416f09 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java
@@ -95,6 +95,11 @@ public class CacheObjectByteArrayImpl implements CacheObject, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectImpl.java
index 694027f..be04d6b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectImpl.java
@@ -110,6 +110,11 @@ public class CacheObjectImpl extends CacheObjectAdapter {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public byte directType() {
         return 89;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryInfo.java
index 6acd09b..c42e788 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryInfo.java
@@ -195,6 +195,11 @@ public class GridCacheEntryInfo implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/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 83e3aa7..b6f5adf 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
@@ -583,6 +583,11 @@ public abstract class GridCacheMessage implements Message {
         }
     }
 
+    /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
     /**
      * @param byteCol Collection to unmarshal.
      * @param ctx Context.

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/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 a9edb95..b3bce09 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
@@ -342,6 +342,11 @@ public class GridCacheReturn implements Externalizable, Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public byte directType() {
         return 88;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/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 4db7ec1..e557c28 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
@@ -74,6 +74,11 @@ public class KeyCacheObjectImpl extends CacheObjectAdapter implements KeyCacheOb
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
         return this;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/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 fec61df..e908c05 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
@@ -1459,6 +1459,9 @@ 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);
                     }
@@ -1515,7 +1518,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         if (remap) {
             assert dhtFut == null;
 
-            res.remapKeys(req.keys());
+            res.remapKeys(keys);
 
             completionCb.apply(req, res);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/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 e417cdb..c8e33c2 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
@@ -28,7 +28,6 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.internal.GridDirectCollection;
 import org.apache.ignite.internal.GridDirectTransient;
-import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
@@ -50,7 +49,6 @@ import org.jetbrains.annotations.Nullable;
 /**
  * Lite dht cache backup update request.
  */
-@IgniteCodeGeneratingFail // Need add 'cleanup' call in 'writeTo' method.
 public class GridDhtAtomicUpdateRequest extends GridCacheMessage implements GridCacheDeployable {
     /** */
     private static final long serialVersionUID = 0L;
@@ -861,8 +859,6 @@ public class GridDhtAtomicUpdateRequest extends GridCacheMessage implements Grid
 
         }
 
-        cleanup();
-
         return true;
     }
 
@@ -1062,6 +1058,11 @@ public class GridDhtAtomicUpdateRequest extends GridCacheMessage implements Grid
         return reader.afterMessageRead(GridDhtAtomicUpdateRequest.class);
     }
 
+    /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        cleanup();
+    }
+
     /**
      * Cleanup values not needed after message was sent.
      */
@@ -1071,8 +1072,9 @@ public class GridDhtAtomicUpdateRequest extends GridCacheMessage implements Grid
 
         // Do not keep values if they are not needed for continuous query notification.
         if (locPrevVals == null) {
-           vals = null;
-           locPrevVals = null;
+            keys = null;
+            vals = null;
+            locPrevVals = null;
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java
index 1a7fa88..0f97e59 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java
@@ -1024,6 +1024,22 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
         return reader.afterMessageRead(GridNearAtomicUpdateRequest.class);
     }
 
+    /**
+     * Cleanup values.
+     *
+     * @param clearKeys If {@code true} clears keys.
+     */
+    public void cleanup(boolean clearKeys) {
+        vals = null;
+        entryProcessors = null;
+        entryProcessorsBytes = null;
+        invokeArgs = null;
+        invokeArgsBytes = null;
+
+        if (clearKeys)
+            keys = null;
+    }
+
     /** {@inheritDoc} */
     @Override public byte directType() {
         return 40;

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
index b4d2558..976405e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
@@ -114,6 +114,11 @@ public class GridDhtPartitionExchangeId implements Message, Comparable<GridDhtPa
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         U.writeUuid(out, nodeId);
         out.writeObject(topVer);

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/CacheVersionedValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/CacheVersionedValue.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/CacheVersionedValue.java
index c14621a..186ffc1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/CacheVersionedValue.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/CacheVersionedValue.java
@@ -98,6 +98,11 @@ public class CacheVersionedValue implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
index e56e445..28a9437 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
@@ -155,6 +155,11 @@ public class GridCacheSqlQuery implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridCacheSqlQuery.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
index 4d3786a..d105271 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
@@ -283,6 +283,11 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public void prepare(GridDeploymentInfo depInfo) {
         this.depInfo = depInfo;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
index 8b871a1..a4fecee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
@@ -952,6 +952,11 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxKey.java
index e346c87..2c07854 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxKey.java
@@ -93,6 +93,11 @@ public class IgniteTxKey implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean equals(Object o) {
         if (this == o)
             return true;

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxEntryValueHolder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxEntryValueHolder.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxEntryValueHolder.java
index ab20f8d..8e2d6bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxEntryValueHolder.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxEntryValueHolder.java
@@ -154,6 +154,11 @@ public class TxEntryValueHolder implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return "[op=" + op +", val=" + val + ']';
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java
index fd695f9..e6cba00 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java
@@ -192,6 +192,11 @@ public class GridCacheVersion implements Message, Comparable<GridCacheVersion>,
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         out.writeInt(topVer);
         out.writeLong(globalTime);

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaSnapshotMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaSnapshotMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaSnapshotMessage.java
index 110e46c..11a0ed7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaSnapshotMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaSnapshotMessage.java
@@ -75,6 +75,11 @@ public class GridClockDeltaSnapshotMessage implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaVersion.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaVersion.java
index 1557806..310fd50 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaVersion.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaVersion.java
@@ -71,6 +71,11 @@ public class GridClockDeltaVersion implements Message, Comparable<GridClockDelta
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public int compareTo(GridClockDeltaVersion o) {
         int res = Long.compare(topVer, o.topVer);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessage.java
index 0ffcd3e..0b629dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessage.java
@@ -144,6 +144,11 @@ public class GridContinuousMessage implements Message {
         this.dataBytes = dataBytes;
     }
 
+    /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
     /**
      * @return Future ID for synchronous event notification.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerEntry.java
index d1c0cc3..c66e08f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerEntry.java
@@ -99,6 +99,11 @@ public class DataStreamerEntry implements Map.Entry<KeyCacheObject, CacheObject>
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerRequest.java
index 77c802d..bd26d58 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerRequest.java
@@ -254,6 +254,11 @@ public class DataStreamerRequest implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(DataStreamerRequest.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerResponse.java
index 4429e56..7d1307a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerResponse.java
@@ -79,6 +79,11 @@ public class DataStreamerResponse implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(DataStreamerResponse.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAckMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAckMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAckMessage.java
index ecb892e..bca4592 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAckMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAckMessage.java
@@ -87,6 +87,11 @@ public class IgfsAckMessage extends IgfsCommunicationMessage {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public void prepareMarshal(Marshaller marsh) throws IgniteCheckedException {
         super.prepareMarshal(marsh);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlockKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlockKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlockKey.java
index a16e60b..06a2e1c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlockKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlockKey.java
@@ -107,6 +107,11 @@ public final class IgfsBlockKey implements Message, Externalizable, Comparable<I
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public int compareTo(@NotNull IgfsBlockKey o) {
         int res = fileId.compareTo(o.fileId);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlocksMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlocksMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlocksMessage.java
index d1f5e4a..8b963fd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlocksMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlocksMessage.java
@@ -85,6 +85,11 @@ public class IgfsBlocksMessage extends IgfsCommunicationMessage {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteMessage.java
index ef3dd43..e59b257 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteMessage.java
@@ -139,6 +139,11 @@ public class IgfsDeleteMessage extends IgfsCommunicationMessage {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
         reader.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileAffinityRange.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileAffinityRange.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileAffinityRange.java
index 9cf1da4..9ac914f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileAffinityRange.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileAffinityRange.java
@@ -236,6 +236,11 @@ public class IgfsFileAffinityRange implements Message, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         U.writeGridUuid(out, affKey);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerRequest.java
index 75d77bd..752c913 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerRequest.java
@@ -75,6 +75,11 @@ public class IgfsFragmentizerRequest extends IgfsCommunicationMessage {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsFragmentizerRequest.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerResponse.java
index b0f8a6b..afb07bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerResponse.java
@@ -55,6 +55,11 @@ public class IgfsFragmentizerResponse extends IgfsCommunicationMessage {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSyncMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSyncMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSyncMessage.java
index cfd8c68..ff9660a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSyncMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSyncMessage.java
@@ -72,6 +72,11 @@ public class IgfsSyncMessage extends IgfsCommunicationMessage {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryCancelRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryCancelRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryCancelRequest.java
index e7e1493..ecc0abd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryCancelRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryCancelRequest.java
@@ -55,6 +55,11 @@ public class GridQueryCancelRequest implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridQueryCancelRequest.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java
index 5f47649..499438d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java
@@ -67,6 +67,11 @@ public class GridQueryFailResponse implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridQueryFailResponse.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageRequest.java
index 5ed6502..84cb57e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageRequest.java
@@ -85,6 +85,11 @@ public class GridQueryNextPageRequest implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java
index ade7d90..b220291 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java
@@ -149,6 +149,11 @@ public class GridQueryNextPageResponse implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java
index a7b4d7d..60d348b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java
@@ -175,6 +175,11 @@ public class GridQueryRequest implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultRequest.java
index 64eb5fc..19a8597 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultRequest.java
@@ -100,6 +100,11 @@ public class GridTaskResultRequest implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultResponse.java
index 8876d4c..a542432 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultResponse.java
@@ -118,6 +118,11 @@ public class GridTaskResultResponse implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/util/GridByteArrayList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridByteArrayList.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridByteArrayList.java
index 5c236d2..e1469e8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridByteArrayList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridByteArrayList.java
@@ -399,6 +399,11 @@ public class GridByteArrayList implements Message, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         out.writeInt(size);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/util/GridLongList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridLongList.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridLongList.java
index 9ef6abb..a8b9bf3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridLongList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridLongList.java
@@ -513,6 +513,11 @@ public class GridLongList implements Message, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/util/GridMessageCollection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridMessageCollection.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridMessageCollection.java
index 6044bbe..9220037 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridMessageCollection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridMessageCollection.java
@@ -82,6 +82,11 @@ public final class GridMessageCollection<M extends Message> implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/util/UUIDCollectionMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/UUIDCollectionMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/util/UUIDCollectionMessage.java
index 33bce53..7773e7b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/UUIDCollectionMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/UUIDCollectionMessage.java
@@ -76,6 +76,11 @@ public class UUIDCollectionMessage implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/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 34c3670..3d18ab7 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
@@ -64,6 +64,11 @@ public class GridNioFinishedFuture<R> extends GridFinishedFuture<R> implements G
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public IgniteInClosure<IgniteException> ackClosure() {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/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 d81760d..b02acc8 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,18 @@ public interface GridNioFuture<R> extends IgniteInternalFuture<R> {
     public boolean skipRecovery();
 
     /**
-     * Sets ack closure which will be applied when ack recevied.
+     * Sets ack closure which will be applied when ack received.
      *
      * @param closure Ack closure.
      */
     public void ackClosure(IgniteInClosure<IgniteException> closure);
 
     /**
+     * The method will be called when ack received.
+     */
+    public void onAckReceived();
+
+    /**
      * @return Ack closure.
      */
     public IgniteInClosure<IgniteException> ackClosure();

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/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 0fc6841..fe97039 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
@@ -56,6 +56,11 @@ public class GridNioFutureImpl<R> extends GridFutureAdapter<R> implements GridNi
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public IgniteInClosure<IgniteException> ackClosure() {
         return ackClosure;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
index 429f990..685d260 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
@@ -180,6 +180,8 @@ public class GridNioRecoveryDescriptor {
             if (fut.ackClosure() != null)
                 fut.ackClosure().apply(null);
 
+            fut.onAckReceived();
+
             acked++;
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/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 75fa9f2..765b139 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
@@ -2176,6 +2176,13 @@ public class GridNioServer<T> {
         }
 
         /** {@inheritDoc} */
+        @Override public void onAckReceived() {
+            assert commMsg != null;
+
+            commMsg.onAckReceived();
+        }
+
+        /** {@inheritDoc} */
         @Override public boolean skipRecovery() {
             return skipRecovery;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/Message.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/Message.java b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/Message.java
index 67f3351..acf0dbb 100644
--- a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/Message.java
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/Message.java
@@ -55,4 +55,9 @@ public interface Message extends Serializable {
      * @return Fields count.
      */
     public byte fieldsCount();
+
+    /**
+     * Method called when ack message received.
+     */
+    public void onAckReceived();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingRequest.java b/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingRequest.java
index 637c12b..1c1e5e7 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingRequest.java
@@ -56,6 +56,11 @@ public class JobStealingRequest implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/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 918bc83..2a078ee 100644
--- 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
@@ -3415,6 +3415,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         }
 
         /** {@inheritDoc} */
+        @Override public void onAckReceived() {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
         @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
             if (buf.remaining() < 33)
                 return false;
@@ -3501,6 +3506,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         }
 
         /** {@inheritDoc} */
+        @Override public void onAckReceived() {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
         @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
             if (buf.remaining() < 9)
                 return false;
@@ -3573,6 +3583,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         }
 
         /** {@inheritDoc} */
+        @Override public void onAckReceived() {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
         @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
             assert nodeIdBytes.length == 16;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java
index 25b637b..9cf14ce 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java
@@ -282,6 +282,11 @@ public class GridAffinityNoCacheSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
+        @Override public void onAckReceived() {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
         @Override public byte directType() {
             throw new UnsupportedOperationException();
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/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 6f762f3..8503b48 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
@@ -154,6 +154,11 @@ public class GridCommunicationSendMessageSelfTest extends GridCommonAbstractTest
         }
 
         /** {@inheritDoc} */
+        @Override public void onAckReceived() {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
         @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
             return true;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerSelfTest.java
index cdd119c..c2cfa76 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerSelfTest.java
@@ -233,6 +233,11 @@ public class GridIoManagerSelfTest extends GridCommonAbstractTest {
     /** */
     private static class TestMessage implements Message {
         /** {@inheritDoc} */
+        @Override public void onAckReceived() {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
         @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
             return true;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterAbstractTest.java
index d8a5006..3df7037 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryCounterAbstractTest.java
@@ -411,9 +411,9 @@ public abstract class CacheContinuousQueryCounterAbstractTest extends GridCommon
                     @Override public void onUpdated(
                         Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
                         for (CacheEntryEvent<? extends Integer, ? extends Integer> e : evts) {
-                            cntr.incrementAndGet();
-
                             synchronized (vals) {
+                                cntr.incrementAndGet();
+
                                 vals.add(new T2<>(e.getValue(),
                                     e.unwrap(CacheQueryEntryEvent.class).getPartitionUpdateCounter()));
                             }
@@ -431,10 +431,9 @@ public abstract class CacheContinuousQueryCounterAbstractTest extends GridCommon
                         }
                     }, 2000L);
 
-                    for (T2<Integer, Long> val : vals) {
-                        assertEquals(vals.size(), keyCnt);
-
-                        assertEquals((long)val.get1() + 1, (long)val.get2());
+                    synchronized (vals) {
+                        for (T2<Integer, Long> val : vals)
+                            assertEquals((long)val.get1() + 1, (long)val.get2());
                     }
                 }
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridTestMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridTestMessage.java b/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridTestMessage.java
index 310dd2b..748ec4a 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridTestMessage.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridTestMessage.java
@@ -94,6 +94,11 @@ class GridTestMessage implements Message, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         U.writeGridUuid(out, id);
         out.writeLong(field1);


[18/21] ignite git commit: IGNITE-2249 - Do not deserialize services on client node

Posted by sb...@apache.org.
IGNITE-2249 - Do not deserialize services on client node


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

Branch: refs/heads/ignite-1232
Commit: 7d5f77e2f9ad80ec298b96452e5f55f737a01701
Parents: 2ad4b5c
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Wed Feb 17 18:21:54 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Feb 17 18:21:54 2016 -0800

----------------------------------------------------------------------
 .../ignite/internal/MarshallerContextImpl.java  |   3 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |   3 +-
 .../CacheDataStructuresManager.java             |   4 +-
 .../continuous/CacheContinuousQueryHandler.java |  21 ++-
 .../continuous/CacheContinuousQueryManager.java |  18 ++-
 .../datastructures/DataStructuresProcessor.java |   1 +
 .../service/GridServiceProcessor.java           | 101 ++++++++++++-
 .../processors/service/GridServiceProxy.java    |  22 +--
 .../GridServiceSerializationSelfTest.java       | 149 +++++++++++++++++++
 .../testsuites/IgniteKernalSelfTestSuite.java   |   2 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |   5 +-
 11 files changed, 287 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7d5f77e2/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
index e3f2bc9..05fe8ef 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
@@ -83,7 +83,8 @@ public class MarshallerContextImpl extends MarshallerContextAdapter {
             new ContinuousQueryListener(ctx.log(MarshallerContextImpl.class), workDir),
             null,
             ctx.cache().marshallerCache().context().affinityNode(),
-            true
+            true,
+            false
         );
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d5f77e2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
index e0da8d1..624a453 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
@@ -277,7 +277,8 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
                 new MetaDataEntryListener(),
                 new MetaDataEntryFilter(),
                 false,
-                true);
+                true,
+                false);
 
             while (true) {
                 ClusterNode oldestSrvNode =

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d5f77e2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
index 47c3dd9..b42e5e7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
@@ -37,7 +37,6 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteSet;
 import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
 import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.cluster.ClusterTopologyException;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
@@ -285,7 +284,8 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
                     },
                     new QueueHeaderPredicate(),
                     cctx.isLocal() || (cctx.isReplicated() && cctx.affinityNode()),
-                    true);
+                    true,
+                    false);
             }
 
             GridCacheQueueProxy queue = queuesMap.get(hdr.id());

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d5f77e2/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 08fe62a..0324e41 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
@@ -151,6 +151,9 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
     /** */
     private AffinityTopologyVersion initTopVer;
 
+    /** */
+    private transient boolean ignoreClassNotFound;
+
     /**
      * Required by {@link Externalizable}.
      */
@@ -188,7 +191,8 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
         int taskHash,
         boolean skipPrimaryCheck,
         boolean locCache,
-        boolean keepBinary) {
+        boolean keepBinary,
+        boolean ignoreClassNotFound) {
         assert topic != null;
         assert locLsnr != null;
 
@@ -205,6 +209,7 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
         this.skipPrimaryCheck = skipPrimaryCheck;
         this.locCache = locCache;
         this.keepBinary = keepBinary;
+        this.ignoreClassNotFound = ignoreClassNotFound;
 
         cacheId = CU.cacheId(cacheName);
     }
@@ -566,6 +571,8 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
 
         final GridCacheContext cctx = cacheContext(ctx);
 
+        Collection<CacheContinuousQueryEntry> entries0 = new ArrayList<>();
+
         for (CacheContinuousQueryEntry e : entries) {
             GridCacheDeploymentManager depMgr = cctx.deploy();
 
@@ -582,19 +589,19 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
 
             try {
                 e.unmarshal(cctx, ldr);
+
+                entries0.addAll(handleEvent(ctx, e));
             }
             catch (IgniteCheckedException ex) {
-                U.error(ctx.log(getClass()), "Failed to unmarshal entry.", ex);
+                if (ignoreClassNotFound)
+                    assert internal;
+                else
+                    U.error(ctx.log(getClass()), "Failed to unmarshal entry.", ex);
             }
         }
 
         final IgniteCache cache = cctx.kernalContext().cache().jcache(cctx.name());
 
-        Collection<CacheContinuousQueryEntry> entries0 = new ArrayList<>();
-
-        for (CacheContinuousQueryEntry e : entries)
-            entries0.addAll(handleEvent(ctx, e));
-
         if (!entries0.isEmpty()) {
             Iterable<CacheEntryEvent<? extends K, ? extends V>> evts = F.viewReadOnly(entries0,
                 new C1<CacheContinuousQueryEntry, CacheEntryEvent<? extends K, ? extends V>>() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d5f77e2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
index 840a61b..409c1da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
@@ -433,7 +433,8 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
             false,
             true,
             loc,
-            keepBinary);
+            keepBinary,
+            false);
     }
 
     /**
@@ -447,7 +448,8 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
     public UUID executeInternalQuery(CacheEntryUpdatedListener<?, ?> locLsnr,
         CacheEntryEventSerializableFilter rmtFilter,
         boolean loc,
-        boolean notifyExisting)
+        boolean notifyExisting,
+        boolean ignoreClassNotFound)
         throws IgniteCheckedException
     {
         return executeQuery0(
@@ -462,7 +464,8 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
             false,
             true,
             loc,
-            false);
+            false,
+            ignoreClassNotFound);
     }
 
     /**
@@ -560,7 +563,8 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
         boolean sync,
         boolean ignoreExpired,
         boolean loc,
-        final boolean keepBinary) throws IgniteCheckedException
+        final boolean keepBinary,
+        boolean ignoreClassNotFound) throws IgniteCheckedException
     {
         cctx.checkSecurity(SecurityPermission.CACHE_READ);
 
@@ -582,7 +586,8 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
             taskNameHash,
             skipPrimaryCheck,
             cctx.isLocal(),
-            keepBinary);
+            keepBinary,
+            ignoreClassNotFound);
 
         IgnitePredicate<ClusterNode> pred = (loc || cctx.config().getCacheMode() == CacheMode.LOCAL) ?
             F.nodeForNodeId(cctx.localNodeId()) : F.<ClusterNode>alwaysTrue();
@@ -790,7 +795,8 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
                 cfg.isSynchronous(),
                 false,
                 false,
-                keepBinary);
+                keepBinary,
+                false);
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d5f77e2/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 98848ee..445fc3e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -248,6 +248,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                     qryId = dsCacheCtx.continuousQueries().executeInternalQuery(new DataStructuresEntryListener(),
                         new DataStructuresEntryFilter(),
                         dsCacheCtx.isReplicated() && dsCacheCtx.affinityNode(),
+                        false,
                         false);
                 }
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d5f77e2/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index 2841083..1a48e8c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -44,8 +44,10 @@ import org.apache.ignite.configuration.DeploymentMode;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.Event;
+import org.apache.ignite.internal.GridClosureCallMode;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
+import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
@@ -58,21 +60,26 @@ import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.query.CacheQuery;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
+import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.X;
 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.LT;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.services.Service;
 import org.apache.ignite.services.ServiceConfiguration;
 import org.apache.ignite.services.ServiceDescriptor;
@@ -166,11 +173,13 @@ public class GridServiceProcessor extends GridProcessorAdapter {
             if (ctx.deploy().enabled())
                 ctx.cache().context().deploy().ignoreOwnership(true);
 
+            boolean affNode = cache.context().affinityNode();
+
             cfgQryId = cache.context().continuousQueries().executeInternalQuery(
-                new DeploymentListener(), null, cache.context().affinityNode(), true);
+                new DeploymentListener(), null, affNode, true, !affNode);
 
             assignQryId = cache.context().continuousQueries().executeInternalQuery(
-                new AssignmentListener(), null, cache.context().affinityNode(), true);
+                new AssignmentListener(), null, affNode, true, !affNode);
         }
         finally {
             if (ctx.deploy().enabled())
@@ -544,6 +553,38 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param name Service name.
+     * @return Service topology.
+     */
+    public Map<UUID, Integer> serviceTopology(String name) throws IgniteCheckedException {
+        ClusterNode node = cache.affinity().mapKeyToNode(name);
+
+        if (node.version().compareTo(ServiceTopologyCallable.SINCE_VER) >= 0) {
+            return ctx.closure().callAsyncNoFailover(
+                GridClosureCallMode.BALANCE,
+                new ServiceTopologyCallable(name),
+                Collections.singletonList(node),
+                false
+            ).get();
+        }
+        else
+            return serviceTopology(cache, name);
+    }
+
+    /**
+     * @param cache Utility cache.
+     * @param svcName Service name.
+     * @return Service topology.
+     * @throws IgniteCheckedException In case of error.
+     */
+    private static Map<UUID, Integer> serviceTopology(IgniteInternalCache<Object, Object> cache, String svcName)
+        throws IgniteCheckedException {
+        GridServiceAssignments val = (GridServiceAssignments)cache.get(new GridServiceAssignmentsKey(svcName));
+
+        return val != null ? val.assigns() : null;
+    }
+
+    /**
      * @return Collection of service descriptors.
      */
     public Collection<ServiceDescriptor> serviceDescriptors() {
@@ -1069,7 +1110,17 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                         if (!(e.getKey() instanceof GridServiceDeploymentKey))
                             continue;
 
-                        GridServiceDeployment dep = (GridServiceDeployment)e.getValue();
+                        GridServiceDeployment dep;
+
+                        try {
+                            dep = (GridServiceDeployment)e.getValue();
+                        }
+                        catch (IgniteException ex) {
+                            if (X.hasCause(ex, ClassNotFoundException.class))
+                                continue;
+                            else
+                                throw ex;
+                        }
 
                         if (dep != null) {
                             svcName.set(dep.configuration().getName());
@@ -1346,7 +1397,17 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                         if (!(e.getKey() instanceof GridServiceAssignmentsKey))
                             continue;
 
-                        GridServiceAssignments assigns = (GridServiceAssignments)e.getValue();
+                        GridServiceAssignments assigns;
+
+                        try {
+                            assigns = (GridServiceAssignments)e.getValue();
+                        }
+                        catch (IgniteException ex) {
+                            if (X.hasCause(ex, ClassNotFoundException.class))
+                                continue;
+                            else
+                                throw ex;
+                        }
 
                         if (assigns != null) {
                             svcName.set(assigns.name());
@@ -1467,4 +1528,34 @@ public class GridServiceProcessor extends GridProcessorAdapter {
             return S.toString(ServiceAssignmentsPredicate.class, this);
         }
     }
-}
\ No newline at end of file
+
+    /**
+     */
+    @GridInternal
+    private static class ServiceTopologyCallable implements IgniteCallable<Map<UUID, Integer>> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private static final IgniteProductVersion SINCE_VER = IgniteProductVersion.fromString("1.5.7");
+
+        /** */
+        private final String svcName;
+
+        /** */
+        @IgniteInstanceResource
+        private IgniteEx ignite;
+
+        /**
+         * @param svcName Service name.
+         */
+        public ServiceTopologyCallable(String svcName) {
+            this.svcName = svcName;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Map<UUID, Integer> call() throws Exception {
+            return serviceTopology(ignite.context().cache().utilityCache(), svcName);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d5f77e2/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java
index e54ec7b..6bec8ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java
@@ -47,7 +47,6 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.resources.IgniteInstanceResource;
-import org.apache.ignite.services.ServiceDescriptor;
 import org.jsr166.ThreadLocalRandom8;
 
 import static org.apache.ignite.internal.GridClosureCallMode.BALANCE;
@@ -210,7 +209,7 @@ class GridServiceProxy<T> implements Serializable {
          * @param name Service name.
          * @return Node with deployed service or {@code null} if there is no such node.
          */
-        private ClusterNode nodeForService(String name, boolean sticky) {
+        private ClusterNode nodeForService(String name, boolean sticky) throws IgniteCheckedException {
             do { // Repeat if reference to remote node was changed.
                 if (sticky) {
                     ClusterNode curNode = rmtNode.get();
@@ -237,11 +236,11 @@ class GridServiceProxy<T> implements Serializable {
          * @return Local node if it has a given service deployed or randomly chosen remote node,
          * otherwise ({@code null} if given service is not deployed on any node.
          */
-        private ClusterNode randomNodeForService(String name) {
+        private ClusterNode randomNodeForService(String name) throws IgniteCheckedException {
             if (hasLocNode && ctx.service().service(name) != null)
                 return ctx.discovery().localNode();
 
-            Map<UUID, Integer> snapshot = serviceTopology(name);
+            Map<UUID, Integer> snapshot = ctx.service().serviceTopology(name);
 
             if (snapshot == null || snapshot.isEmpty())
                 return null;
@@ -307,19 +306,6 @@ class GridServiceProxy<T> implements Serializable {
 
             return null;
         }
-
-        /**
-         * @param name Service name.
-         * @return Map of number of service instances per node ID.
-         */
-        private Map<UUID, Integer> serviceTopology(String name) {
-            for (ServiceDescriptor desc : ctx.service().serviceDescriptors()) {
-                if (desc.name().equals(name))
-                    return desc.topologySnapshot();
-            }
-
-            return null;
-        }
     }
 
     /**
@@ -403,4 +389,4 @@ class GridServiceProxy<T> implements Serializable {
             return S.toString(ServiceProxyCallable.class, this);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d5f77e2/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceSerializationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceSerializationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceSerializationSelfTest.java
new file mode 100644
index 0000000..f709dfe
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceSerializationSelfTest.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.service;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.services.Service;
+import org.apache.ignite.services.ServiceContext;
+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.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.thread.IgniteThread;
+
+/**
+ * Service serialization test.
+ */
+public class GridServiceSerializationSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER));
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testServiceSerialization() throws Exception {
+        try {
+            Ignite server = startGridsMultiThreaded(3);
+
+            Ignition.setClientMode(true);
+
+            Ignite client = startGrid("client");
+
+            server.services(server.cluster().forServers())
+                .deployClusterSingleton("my-service", new MyServiceImpl());
+
+            MyService svc = client.services().serviceProxy("my-service", MyService.class, false);
+
+            svc.hello();
+
+            assert MyServiceImpl.latch.await(2000, TimeUnit.MILLISECONDS);
+
+            assertEquals(0, MyServiceImpl.cnt.get());
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     */
+    private static interface MyService extends Service {
+        /** */
+        void hello();
+    }
+
+    /**
+     */
+    private static class MyServiceImpl implements MyService, Externalizable {
+        /** */
+        static final AtomicInteger cnt = new AtomicInteger();
+
+        /** */
+        static final CountDownLatch latch = new CountDownLatch(1);
+
+        /**
+         */
+        public MyServiceImpl() throws ClassNotFoundException {
+            if (clientThread())
+                throw new ClassNotFoundException("Expected ClassNotFoundException");
+        }
+
+        /** {@inheritDoc} */
+        @Override public void cancel(ServiceContext ctx) {
+            if (clientThread())
+                cnt.incrementAndGet();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void init(ServiceContext ctx) throws Exception {
+            if (clientThread())
+                cnt.incrementAndGet();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void execute(ServiceContext ctx) throws Exception {
+            if (clientThread())
+                cnt.incrementAndGet();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void hello() {
+            if (clientThread())
+                cnt.incrementAndGet();
+
+            latch.countDown();
+        }
+
+        /**
+         * @return If current thread belongs to client.
+         */
+        private boolean clientThread() {
+            assert Thread.currentThread() instanceof IgniteThread;
+
+            return ((IgniteThread)Thread.currentThread()).getGridName().contains("client");
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d5f77e2/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
index d9e9b0f..214d375 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
@@ -54,6 +54,7 @@ import org.apache.ignite.internal.processors.service.GridServiceProcessorProxySe
 import org.apache.ignite.internal.processors.service.GridServiceProcessorSingleNodeSelfTest;
 import org.apache.ignite.internal.processors.service.GridServiceProcessorStopSelfTest;
 import org.apache.ignite.internal.processors.service.GridServiceReassignmentSelfTest;
+import org.apache.ignite.internal.processors.service.GridServiceSerializationSelfTest;
 import org.apache.ignite.internal.processors.service.ServicePredicateAccessCacheTest;
 import org.apache.ignite.internal.util.GridStartupWithSpecifiedWorkDirectorySelfTest;
 import org.apache.ignite.internal.util.GridStartupWithUndefinedIgniteHomeSelfTest;
@@ -123,6 +124,7 @@ public class IgniteKernalSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridServiceProcessorStopSelfTest.class);
         suite.addTestSuite(ServicePredicateAccessCacheTest.class);
         suite.addTestSuite(GridServicePackagePrivateSelfTest.class);
+        suite.addTestSuite(GridServiceSerializationSelfTest.class);
 
         return suite;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d5f77e2/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
index 81ff8ea..f4cf892 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
@@ -255,7 +255,8 @@ public class HadoopJobTracker extends HadoopComponent {
             },
             null,
             true,
-            true
+            true,
+            false
         );
 
         ctx.kernalContext().event().addLocalEventListener(new GridLocalEventListener() {
@@ -1690,4 +1691,4 @@ public class HadoopJobTracker extends HadoopComponent {
          */
         protected abstract void update(HadoopJobMetadata meta, HadoopJobMetadata cp);
     }
-}
\ No newline at end of file
+}


[20/21] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-1232

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/master' into ignite-1232

Conflicts:
	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
	modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java


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

Branch: refs/heads/ignite-1232
Commit: 97cd7830820bc29ef61dff2e2751ae84199a65e9
Parents: 86bddb9 21f22d3
Author: sboikov <sb...@gridgain.com>
Authored: Thu Feb 18 09:03:51 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Feb 18 09:03:51 2016 +0300

----------------------------------------------------------------------
 .../cache/query/CacheQueryEntryEvent.java       |  48 +
 .../ignite/internal/GridJobCancelRequest.java   |   5 +
 .../ignite/internal/GridJobExecuteRequest.java  |   5 +
 .../ignite/internal/GridJobExecuteResponse.java |   5 +
 .../ignite/internal/GridJobSiblingsRequest.java |   5 +
 .../internal/GridJobSiblingsResponse.java       |   5 +
 .../ignite/internal/GridTaskCancelRequest.java  |   5 +
 .../ignite/internal/GridTaskSessionRequest.java |   5 +
 .../ignite/internal/MarshallerContextImpl.java  |   3 +-
 .../ignite/internal/binary/BinaryContext.java   |   1 +
 .../internal/binary/BinaryEnumObjectImpl.java   |   5 +
 .../internal/binary/BinaryObjectImpl.java       |   5 +
 .../binary/BinaryObjectOffheapImpl.java         |   5 +
 .../ignite/internal/binary/BinaryUtils.java     |   5 +-
 .../checkpoint/GridCheckpointRequest.java       |   5 +
 .../managers/communication/GridIoMessage.java   |   5 +
 .../communication/GridIoUserMessage.java        |   5 +
 .../deployment/GridDeploymentInfoBean.java      |   5 +
 .../deployment/GridDeploymentRequest.java       |   5 +
 .../deployment/GridDeploymentResponse.java      |   5 +
 .../eventstorage/GridEventStorageMessage.java   |   5 +
 .../affinity/AffinityTopologyVersion.java       |   5 +
 .../cache/CacheEntryInfoCollection.java         |   5 +
 .../cache/CacheEntryPredicateAdapter.java       |   5 +
 .../cache/CacheEntryPredicateContainsValue.java |   8 +-
 .../cache/CacheEntrySerializablePredicate.java  |   5 +
 .../cache/CacheEvictableEntryImpl.java          |   2 +-
 .../processors/cache/CacheEvictionEntry.java    |   5 +
 .../cache/CacheInvokeDirectResult.java          |   5 +
 .../cache/CacheObjectByteArrayImpl.java         |   5 +
 .../processors/cache/CacheObjectImpl.java       |   5 +
 .../processors/cache/GridCacheEntryInfo.java    |   5 +
 .../processors/cache/GridCacheMessage.java      |   5 +
 .../processors/cache/GridCacheReturn.java       |   5 +
 .../processors/cache/KeyCacheObjectImpl.java    |   5 +
 .../binary/CacheObjectBinaryProcessorImpl.java  |  13 +-
 .../CacheDataStructuresManager.java             |   4 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   5 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |  14 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |  16 +
 .../dht/preloader/GridDhtPartitionDemander.java |   3 +-
 .../preloader/GridDhtPartitionExchangeId.java   |   5 +
 .../distributed/near/CacheVersionedValue.java   |   5 +
 .../cache/query/GridCacheSqlQuery.java          |   5 +
 .../continuous/CacheContinuousQueryEntry.java   |   5 +
 .../continuous/CacheContinuousQueryEvent.java   |  17 +-
 .../continuous/CacheContinuousQueryHandler.java | 120 +--
 .../CacheContinuousQueryListener.java           |   3 +-
 .../continuous/CacheContinuousQueryManager.java |  65 +-
 .../cache/transactions/IgniteTxEntry.java       |   5 +
 .../cache/transactions/IgniteTxKey.java         |   5 +
 .../cache/transactions/TxEntryValueHolder.java  |   5 +
 .../cache/version/GridCacheVersion.java         |   5 +
 .../clock/GridClockDeltaSnapshotMessage.java    |   5 +
 .../processors/clock/GridClockDeltaVersion.java |   5 +
 .../continuous/GridContinuousMessage.java       |   5 +
 .../datastreamer/DataStreamerEntry.java         |   5 +
 .../datastreamer/DataStreamerRequest.java       |   5 +
 .../datastreamer/DataStreamerResponse.java      |   5 +
 .../datastructures/DataStructuresProcessor.java |   1 +
 .../processors/igfs/IgfsAckMessage.java         |   5 +
 .../internal/processors/igfs/IgfsBlockKey.java  |   5 +
 .../processors/igfs/IgfsBlocksMessage.java      |   5 +
 .../processors/igfs/IgfsDeleteMessage.java      |   5 +
 .../processors/igfs/IgfsFileAffinityRange.java  |   5 +
 .../igfs/IgfsFragmentizerRequest.java           |   5 +
 .../igfs/IgfsFragmentizerResponse.java          |   5 +
 .../processors/igfs/IgfsSyncMessage.java        |   5 +
 .../platform/PlatformProcessorImpl.java         |  17 +-
 .../callback/PlatformCallbackGateway.java       |  30 +
 .../callback/PlatformCallbackUtils.java         |  15 +
 .../messages/GridQueryCancelRequest.java        |   5 +
 .../twostep/messages/GridQueryFailResponse.java |   5 +
 .../messages/GridQueryNextPageRequest.java      |   5 +
 .../messages/GridQueryNextPageResponse.java     |   5 +
 .../h2/twostep/messages/GridQueryRequest.java   |   5 +
 .../handlers/task/GridTaskResultRequest.java    |   5 +
 .../handlers/task/GridTaskResultResponse.java   |   5 +
 .../service/GridServiceProcessor.java           | 101 ++-
 .../processors/service/GridServiceProxy.java    |  22 +-
 .../ignite/internal/util/GridByteArrayList.java |   5 +
 .../ignite/internal/util/GridLongList.java      |   5 +
 .../internal/util/GridMessageCollection.java    |   5 +
 .../internal/util/UUIDCollectionMessage.java    |   5 +
 .../util/nio/GridNioFinishedFuture.java         |   5 +
 .../ignite/internal/util/nio/GridNioFuture.java |   7 +-
 .../internal/util/nio/GridNioFutureImpl.java    |   5 +
 .../util/nio/GridNioRecoveryDescriptor.java     |   2 +
 .../ignite/internal/util/nio/GridNioServer.java |   7 +
 .../extensions/communication/Message.java       |   5 +
 .../jobstealing/JobStealingRequest.java         |   5 +
 .../communication/tcp/TcpCommunicationSpi.java  |  15 +
 .../internal/GridAffinityNoCacheSelfTest.java   |   5 +
 .../GridCommunicationSendMessageSelfTest.java   |   5 +
 .../communication/GridIoManagerSelfTest.java    |   5 +
 .../cache/CacheEnumOperationsAbstractTest.java  | 307 +++++++
 .../CacheEnumOperationsSingleNodeTest.java      |  28 +
 .../cache/CacheEnumOperationsTest.java          |  28 +
 .../IgniteCacheNearRestartRollbackSelfTest.java |  36 +-
 .../GridCacheEvictableEntryEqualsSelfTest.java  |  85 ++
 .../continuous/CacheContinuousBatchAckTest.java | 355 ++++++++
 ...heContinuousBatchForceServerModeAckTest.java |  80 ++
 ...CacheContinuousQueryCounterAbstractTest.java | 612 +++++++++++++
 ...inuousQueryCounterPartitionedAtomicTest.java |  41 +
 ...ContinuousQueryCounterPartitionedTxTest.java |  41 +
 ...tinuousQueryCounterReplicatedAtomicTest.java |  41 +
 ...eContinuousQueryCounterReplicatedTxTest.java |  41 +
 .../CacheContinuousQueryLostPartitionTest.java  |   4 +-
 ...acheContinuousQueryRandomOperationsTest.java | 896 ++++++++++++++++---
 .../GridServiceSerializationSelfTest.java       | 149 +++
 .../communication/GridTestMessage.java          |   5 +
 .../spi/communication/GridTestMessage.java      |   5 +
 .../IgniteCacheEvictionSelfTestSuite.java       |   4 +-
 .../testsuites/IgniteCacheTestSuite2.java       |   4 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |   2 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |   5 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  96 +-
 .../h2/twostep/msg/GridH2ValueMessage.java      |   5 +
 .../h2/GridIndexingSpiAbstractSelfTest.java     |  43 +
 .../IgniteCacheQuerySelfTestSuite.java          |  12 +
 .../cpp/common/include/ignite/common/exports.h  |   6 +-
 .../cpp/common/include/ignite/common/java.h     |  21 +-
 modules/platforms/cpp/common/src/exports.cpp    |   6 +-
 modules/platforms/cpp/common/src/java.cpp       |  28 +-
 .../cpp/core-test/src/cache_query_test.cpp      |  45 +
 .../ignite/cache/query/query_fields_row.h       |   2 +-
 .../Apache.Ignite.Core.Tests.csproj             |   2 +
 .../Process/IgniteProcess.cs                    |  16 +
 .../ProcessExtensions.cs                        |  78 ++
 .../Apache.Ignite.Core.Tests/ReconnectTest.cs   |  96 ++
 .../Apache.Ignite.Core.csproj                   |   1 +
 .../Apache.Ignite.Core/Cluster/ICluster.cs      |  14 +
 .../Common/ClientDisconnectedException.cs       |  97 ++
 .../Impl/Binary/BinaryUtils.cs                  |   2 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |   2 +-
 .../Impl/Compute/ComputeTaskHolder.cs           |   2 +-
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   | 110 ++-
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  32 +-
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |   7 +
 .../Impl/Unmanaged/UnmanagedCallbackHandlers.cs |   3 +
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        |  32 +-
 pom.xml                                         |   2 +-
 142 files changed, 3989 insertions(+), 429 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/97cd7830/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/97cd7830/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/97cd7830/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryCancelRequest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/97cd7830/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/97cd7830/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageRequest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/97cd7830/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/97cd7830/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 2d437ed,288c2b3..cb41250
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@@ -1749,12 -1576,13 +1763,12 @@@ public class IgniteH2Indexing implement
      }
  
      /** {@inheritDoc} */
-     @Override public void registerCache(GridCacheContext<?,?> cctx, CacheConfiguration<?,?> ccfg) 
 -    @Override public void registerCache(CacheConfiguration<?,?> ccfg) throws IgniteCheckedException {
++    @Override public void registerCache(GridCacheContext<?,?> cctx, CacheConfiguration<?,?> ccfg)
 +        throws IgniteCheckedException {
          String schema = schemaNameFromCacheConf(ccfg);
  
 -        if (schemas.putIfAbsent(schema, new Schema(ccfg.getName(), schema,
 -            ccfg.getOffHeapMaxMemory() >= 0 || ccfg.getMemoryMode() == CacheMemoryMode.OFFHEAP_TIERED ?
 -            new GridUnsafeMemory(0) : null, ccfg)) != null)
 -            throw new IgniteCheckedException("Schema for cache already registered: " + U.maskName(ccfg.getName()));
 +        if (schemas.putIfAbsent(schema, new Schema(ccfg.getName(), schema, cctx, ccfg)) != null)
 +            throw new IgniteCheckedException("Cache already registered: " + U.maskName(ccfg.getName()));
  
          space2schema.put(emptyIfNull(ccfg.getName()), schema);
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/97cd7830/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessage.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/97cd7830/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index fd53e40,0da71c8..bb96cd8
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@@ -218,14 -218,33 +218,33 @@@ public abstract class GridIndexingSpiAb
          spi.registerType(typeBA.space(), typeBA);
  
          // Initially all is empty.
 -        assertEquals(0, spi.size(typeAA.space(), typeAA, null));
 -        assertEquals(0, spi.size(typeAB.space(), typeAB, null));
 -        assertEquals(0, spi.size(typeBA.space(), typeBA, null));
 +        assertEquals(0, spi.size(typeAA.space(), typeAA));
 +        assertEquals(0, spi.size(typeAB.space(), typeAB));
 +        assertEquals(0, spi.size(typeBA.space(), typeBA));
  
 -        assertFalse(spi.query(typeAA.space(), "select * from A.A", Collections.emptySet(), typeAA, null).hasNext());
 -        assertFalse(spi.query(typeAB.space(), "select * from A.B", Collections.emptySet(), typeAB, null).hasNext());
 -        assertFalse(spi.query(typeBA.space(), "select * from B.A", Collections.emptySet(), typeBA, null).hasNext());
 +        assertFalse(spi.queryLocalSql(typeAA.space(), "select * from A.A", Collections.emptySet(), typeAA, null).hasNext());
 +        assertFalse(spi.queryLocalSql(typeAB.space(), "select * from A.B", Collections.emptySet(), typeAB, null).hasNext());
 +        assertFalse(spi.queryLocalSql(typeBA.space(), "select * from B.A", Collections.emptySet(), typeBA, null).hasNext());
  
+         assertFalse(spi.query(typeBA.space(), "select * from B.A, A.B, A.A",
+             Collections.emptySet(), typeBA, null).hasNext());
+ 
+         try {
+             spi.query(typeBA.space(), "select aa.*, ab.*, ba.* from A.A aa, A.B ab, B.A ba",
+                 Collections.emptySet(), typeBA, null).hasNext();
+ 
+             fail("Enumerations of aliases in select block must be prohibited");
+         }
+         catch (IgniteCheckedException e) {
+             // all fine
+         }
+ 
+         assertFalse(spi.query(typeAB.space(), "select ab.* from A.B ab",
+             Collections.emptySet(), typeAB, null).hasNext());
+ 
+         assertFalse(spi.query(typeBA.space(), "select   ba.*   from B.A  as ba",
+             Collections.emptySet(), typeBA, null).hasNext());
+ 
          // Nothing to remove.
          spi.remove("A", key(1), aa(1, "", 10));
          spi.remove("B", key(1), ba(1, "", 10, true));
@@@ -286,7 -305,16 +305,16 @@@
          assertEquals(aa(2, "Valera", 19).value(null, false), value(res.next()));
          assertFalse(res.hasNext());
  
 -        res = spi.query(typeAA.space(), "select aa.* from a aa order by aa.age",
++        res = spi.queryLocalSql(typeAA.space(), "select aa.* from a aa order by aa.age",
+             Collections.emptySet(), typeAA, null);
+ 
+         assertTrue(res.hasNext());
+         assertEquals(aa(3, "Borya", 18).value(null, false), value(res.next()));
+         assertTrue(res.hasNext());
+         assertEquals(aa(2, "Valera", 19).value(null, false), value(res.next()));
+         assertFalse(res.hasNext());
+ 
 -        res = spi.query(typeAB.space(), "from b order by name", Collections.emptySet(), typeAB, null);
 +        res = spi.queryLocalSql(typeAB.space(), "from b order by name", Collections.emptySet(), typeAB, null);
  
          assertTrue(res.hasNext());
          assertEquals(ab(1, "Vasya", 20, "Some text about Vasya goes here.").value(null, false), value(res.next()));
@@@ -294,7 -322,16 +322,17 @@@
          assertEquals(ab(4, "Vitalya", 20, "Very Good guy").value(null, false), value(res.next()));
          assertFalse(res.hasNext());
  
 -        res = spi.query(typeAB.space(), "select bb.* from b as bb order by bb.name",
++        res = spi.queryLocalSql(typeAB.space(), "select bb.* from b as bb order by bb.name",
+             Collections.emptySet(), typeAB, null);
+ 
+         assertTrue(res.hasNext());
+         assertEquals(ab(1, "Vasya", 20, "Some text about Vasya goes here.").value(null, false), value(res.next()));
+         assertTrue(res.hasNext());
+         assertEquals(ab(4, "Vitalya", 20, "Very Good guy").value(null, false), value(res.next()));
+         assertFalse(res.hasNext());
+ 
 -        res = spi.query(typeBA.space(), "from a", Collections.emptySet(), typeBA, null);
++
 +        res = spi.queryLocalSql(typeBA.space(), "from a", Collections.emptySet(), typeBA, null);
  
          assertTrue(res.hasNext());
          assertEquals(ba(2, "Kolya", 25, true).value(null, false), value(res.next()));

http://git-wip-us.apache.org/repos/asf/ignite/blob/97cd7830/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index b2b1c61,457ab9b..6ed4004
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@@ -210,7 -224,15 +220,9 @@@ public class IgniteCacheQuerySelfTestSu
          suite.addTestSuite(CacheContinuousQueryFailoverAtomicPrimaryWriteOrderOffheapTieredTest.class);
          suite.addTestSuite(CacheContinuousQueryFailoverTxOffheapTieredTest.class);
          suite.addTestSuite(CacheContinuousQueryRandomOperationsTest.class);
+         suite.addTestSuite(CacheContinuousBatchAckTest.class);
+         suite.addTestSuite(CacheContinuousBatchForceServerModeAckTest.class);
  
 -        // Reduce fields queries.
 -        suite.addTestSuite(GridCacheReduceFieldsQueryLocalSelfTest.class);
 -        suite.addTestSuite(GridCacheReduceFieldsQueryPartitionedSelfTest.class);
 -        suite.addTestSuite(GridCacheReduceFieldsQueryAtomicSelfTest.class);
 -        suite.addTestSuite(GridCacheReduceFieldsQueryReplicatedSelfTest.class);
 -
          suite.addTestSuite(GridCacheQueryIndexingDisabledSelfTest.class);
  
          suite.addTestSuite(GridCacheSwapScanQuerySelfTest.class);


[10/21] ignite git commit: ignite-2669 Changed BinaryUtils.isBinaryType to always convert enums to BinaryObject

Posted by sb...@apache.org.
ignite-2669 Changed BinaryUtils.isBinaryType to always convert enums to BinaryObject


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

Branch: refs/heads/ignite-1232
Commit: 8be62f0281bfe1af27d1e47b1125461644e3c898
Parents: 8562b00
Author: sboikov <sb...@gridgain.com>
Authored: Wed Feb 17 14:57:27 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Feb 17 14:57:27 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/binary/BinaryUtils.java     |   5 +-
 .../cache/CacheEntryPredicateContainsValue.java |   8 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |  10 +-
 .../cache/CacheEnumOperationsAbstractTest.java  | 307 +++++++++++++++++++
 .../CacheEnumOperationsSingleNodeTest.java      |  28 ++
 .../cache/CacheEnumOperationsTest.java          |  28 ++
 .../testsuites/IgniteCacheTestSuite2.java       |   4 +
 7 files changed, 379 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8be62f02/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
index 8b5d780..1b53ffd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
@@ -584,10 +584,7 @@ public class BinaryUtils {
     public static boolean isBinaryType(Class<?> cls) {
         assert cls != null;
 
-        return BinaryObject.class.isAssignableFrom(cls) ||
-            BINARY_CLS.contains(cls) ||
-            cls.isEnum() ||
-            (cls.isArray() && cls.getComponentType().isEnum());
+        return BinaryObject.class.isAssignableFrom(cls) || BINARY_CLS.contains(cls);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/8be62f02/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateContainsValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateContainsValue.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateContainsValue.java
index e89fff4..3db8ae8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateContainsValue.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateContainsValue.java
@@ -58,8 +58,12 @@ public class CacheEntryPredicateContainsValue extends CacheEntryPredicateAdapter
     @Override public boolean apply(GridCacheEntryEx e) {
         CacheObject val = peekVisibleValue(e);
 
-        return F.eq(this.val.value(e.context().cacheObjectContext(), false),
-            CU.value(val, e.context(), false));
+        GridCacheContext cctx = e.context();
+
+        Object thisVal = CU.value(this.val, cctx, false);
+        Object cacheVal = CU.value(val, cctx, false);
+
+        return F.eq(thisVal, cacheVal);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/8be62f02/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
index 5a72a40..e0da8d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
@@ -514,7 +514,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
         Object obj0 = binaryMarsh.unmarshal(arr, null);
 
         // Possible if a class has writeObject method.
-        if (obj0 instanceof BinaryObject)
+        if (obj0 instanceof BinaryObjectImpl)
             ((BinaryObjectImpl)obj0).detachAllowed(true);
 
         return obj0;
@@ -782,8 +782,8 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
         if (((CacheObjectBinaryContext)ctx).binaryEnabled()) {
             obj = toBinary(obj);
 
-            if (obj instanceof BinaryObject)
-                return (BinaryObjectImpl)obj;
+            if (obj instanceof KeyCacheObject)
+                return (KeyCacheObject)obj;
         }
 
         return toCacheKeyObject0(obj, userObj);
@@ -800,8 +800,8 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
         obj = toBinary(obj);
 
-        if (obj instanceof BinaryObject)
-            return (BinaryObjectImpl)obj;
+        if (obj instanceof CacheObject)
+            return (CacheObject)obj;
 
         return toCacheObject0(obj, userObj);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8be62f02/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
new file mode 100644
index 0000000..3957e46
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
@@ -0,0 +1,307 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+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.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.PRIMARY;
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_VALUES;
+import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ *
+ */
+public abstract class CacheEnumOperationsAbstractTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /**
+     * @return Number of nodes.
+     */
+    protected abstract boolean singleNode();
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        if (!singleNode()) {
+            startGridsMultiThreaded(4);
+
+            client = true;
+
+            startGridsMultiThreaded(4, 2);
+        }
+        else
+            startGrid(0);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomic() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC, ONHEAP_TIERED);
+
+        enumOperations(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicOffheapValues() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC, OFFHEAP_VALUES);
+
+        enumOperations(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicOffheapTiered() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC, OFFHEAP_TIERED);
+
+        enumOperations(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTx() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC, ONHEAP_TIERED);
+
+        enumOperations(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxOffheapValues() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC, OFFHEAP_VALUES);
+
+        enumOperations(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxOffheapTiered() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC, OFFHEAP_TIERED);
+
+        enumOperations(ccfg);
+    }
+
+    /**
+     * @param ccfg Cache configuration.
+     */
+    private void enumOperations(CacheConfiguration<Object, Object> ccfg) {
+        ignite(0).createCache(ccfg);
+
+        try {
+            int key = 0;
+
+            int nodes;
+
+            if (!singleNode()) {
+                nodes = 6;
+
+                ignite(nodes - 1).createNearCache(ccfg.getName(), new NearCacheConfiguration<>());
+            }
+            else
+                nodes = 1;
+
+            for (int i = 0; i < nodes; i++) {
+                IgniteCache<Object, Object> cache = ignite(i).cache(ccfg.getName());
+
+                for (int j = 0; j < 100; j++)
+                    enumOperations(cache, key++);
+            }
+        }
+        finally {
+            ignite(0).destroyCache(ccfg.getName());
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     * @param key Key.
+     */
+    private void enumOperations(IgniteCache<Object, Object> cache, int key) {
+        assertNull(cache.get(key));
+
+        assertFalse(cache.replace(key, TestEnum.VAL1));
+
+        assertTrue(cache.putIfAbsent(key, TestEnum.VAL1));
+
+        assertEquals(TestEnum.VAL1, cache.get(key));
+
+        assertFalse(cache.putIfAbsent(key, TestEnum.VAL2));
+
+        assertEquals(TestEnum.VAL1, cache.get(key));
+
+        assertTrue(cache.replace(key, TestEnum.VAL2));
+
+        assertEquals(TestEnum.VAL2, cache.get(key));
+
+        assertFalse(cache.replace(key, TestEnum.VAL1, TestEnum.VAL3));
+
+        assertEquals(TestEnum.VAL2, cache.get(key));
+
+        assertTrue(cache.replace(key, TestEnum.VAL2, TestEnum.VAL3));
+
+        assertEquals(TestEnum.VAL3, cache.get(key));
+
+        assertEquals(TestEnum.VAL3, cache.getAndPut(key, TestEnum.VAL1));
+
+        assertEquals(TestEnum.VAL1, cache.get(key));
+
+        assertEquals(TestEnum.VAL1, cache.invoke(key, new EnumProcessor(TestEnum.VAL2, TestEnum.VAL1)));
+
+        assertEquals(TestEnum.VAL2, cache.get(key));
+
+        assertEquals(TestEnum.VAL2, cache.getAndReplace(key, TestEnum.VAL3));
+
+        assertEquals(TestEnum.VAL3, cache.get(key));
+
+        assertEquals(TestEnum.VAL3, cache.getAndPutIfAbsent(key, TestEnum.VAL1));
+
+        assertEquals(TestEnum.VAL3, cache.get(key));
+
+        cache.put(key, TestEnum.VAL1);
+
+        assertEquals(TestEnum.VAL1, cache.get(key));
+
+        assertEquals(TestEnum.VAL1, cache.getAndRemove(key));
+
+        assertNull(cache.get(key));
+
+        assertFalse(cache.replace(key, TestEnum.VAL2, TestEnum.VAL3));
+
+        assertNull(cache.getAndPutIfAbsent(key, TestEnum.VAL1));
+
+        assertEquals(TestEnum.VAL1, cache.get(key));
+    }
+
+    /**
+     *
+     * @param cacheMode Cache mode.
+     * @param backups Number of backups.
+     * @param atomicityMode Cache atomicity mode.
+     * @param memoryMode Cache memory mode.
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<Object, Object> cacheConfiguration(
+        CacheMode cacheMode,
+        int backups,
+        CacheAtomicityMode atomicityMode,
+        CacheMemoryMode memoryMode) {
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
+
+        ccfg.setAtomicityMode(atomicityMode);
+        ccfg.setCacheMode(cacheMode);
+        ccfg.setMemoryMode(memoryMode);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setAtomicWriteOrderMode(PRIMARY);
+
+        if (memoryMode == OFFHEAP_TIERED)
+            ccfg.setOffHeapMaxMemory(0);
+
+        if (cacheMode == PARTITIONED)
+            ccfg.setBackups(backups);
+
+        return ccfg;
+    }
+
+    /**
+     *
+     */
+    public enum TestEnum {
+        /** */
+        VAL1,
+        /** */
+        VAL2,
+        /** */
+        VAL3
+    }
+
+    /**
+     *
+     */
+    static class EnumProcessor implements EntryProcessor<Object, Object, Object> {
+        /** */
+        private TestEnum newVal;
+
+        /** */
+        private TestEnum expOldVal;
+
+        /**
+         * @param newVal New value.
+         * @param expOldVal Expected old value.
+         */
+        public EnumProcessor(TestEnum newVal, TestEnum expOldVal) {
+            this.newVal = newVal;
+            this.expOldVal = expOldVal;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object process(MutableEntry<Object, Object> entry, Object... args) {
+            TestEnum val = (TestEnum)entry.getValue();
+
+            assertEquals(expOldVal, val);
+
+            entry.setValue(newVal);
+
+            return val;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8be62f02/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsSingleNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsSingleNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsSingleNodeTest.java
new file mode 100644
index 0000000..baaaac3
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsSingleNodeTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+/**
+ *
+ */
+public class CacheEnumOperationsSingleNodeTest extends CacheEnumOperationsAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected boolean singleNode() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8be62f02/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsTest.java
new file mode 100644
index 0000000..8e4964f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+/**
+ *
+ */
+public class CacheEnumOperationsTest extends CacheEnumOperationsAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected boolean singleNode() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8be62f02/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index d83b272..3e8d66a 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -24,6 +24,8 @@ import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunctionBac
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunctionExcludeNeighborsSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheConfigurationLeakTest;
 import org.apache.ignite.internal.processors.cache.CacheDhtLocalPartitionAfterRemoveSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheEnumOperationsSingleNodeTest;
+import org.apache.ignite.internal.processors.cache.CacheEnumOperationsTest;
 import org.apache.ignite.internal.processors.cache.CrossCacheTxRandomOperationsTest;
 import org.apache.ignite.internal.processors.cache.GridCacheAtomicMessageCountSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheFinishPartitionsSelfTest;
@@ -250,6 +252,8 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(CacheLockReleaseNodeLeaveTest.class));
         suite.addTest(new TestSuite(NearCacheSyncUpdateTest.class));
         suite.addTest(new TestSuite(CacheConfigurationLeakTest.class));
+        suite.addTest(new TestSuite(CacheEnumOperationsSingleNodeTest.class));
+        suite.addTest(new TestSuite(CacheEnumOperationsTest.class));
 
         return suite;
     }


[11/21] ignite git commit: IGNITE-2672

Posted by sb...@apache.org.
IGNITE-2672


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

Branch: refs/heads/ignite-1232
Commit: f0f9ef65cab7231d026b8ea126e5e247515993d0
Parents: 60b6f09
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Feb 17 15:07:52 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Feb 17 15:07:52 2016 +0300

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f0f9ef65/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index bead3ae..ccfd471 100644
--- a/pom.xml
+++ b/pom.xml
@@ -84,7 +84,6 @@
         <module>modules/osgi-paxlogging</module>
         <module>modules/osgi-karaf</module>
         <module>modules/osgi</module>
-		<module>modules/benchmarks</module>
     </modules>
 
     <profiles>
@@ -430,6 +429,7 @@
         <profile>
             <id>benchmarks</id>
             <modules>
+                <module>modules/benchmarks</module>
                 <module>modules/yardstick</module>
             </modules>
         </profile>


[02/21] ignite git commit: Increased missedClientHeartbeats in test to avoid client disconnect.

Posted by sb...@apache.org.
Increased missedClientHeartbeats in test to avoid client disconnect.


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

Branch: refs/heads/ignite-1232
Commit: 60b6f09676574048765deea656eec0dfcf589b78
Parents: 675a7c1
Author: sboikov <sb...@gridgain.com>
Authored: Tue Feb 16 13:50:14 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Feb 16 13:52:39 2016 +0300

----------------------------------------------------------------------
 .../IgniteCacheNearRestartRollbackSelfTest.java | 36 +++++++++++---------
 1 file changed, 19 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/60b6f096/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearRestartRollbackSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearRestartRollbackSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearRestartRollbackSelfTest.java
index 6941bcc..6c27a46 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearRestartRollbackSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearRestartRollbackSelfTest.java
@@ -71,6 +71,7 @@ public class IgniteCacheNearRestartRollbackSelfTest extends GridCommonAbstractTe
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
 
         discoSpi.setIpFinder(IP_FINDER);
+        discoSpi.setMaxMissedClientHeartbeats(50);
 
         cfg.setDiscoverySpi(discoSpi);
 
@@ -153,16 +154,16 @@ public class IgniteCacheNearRestartRollbackSelfTest extends GridCommonAbstractTe
                 }
             });
 
-            int currentValue = 0;
+            int currVal = 0;
             boolean invoke = false;
 
             while (!fut.isDone()) {
-                updateCache(tester, currentValue, invoke, false, keys);
+                updateCache(tester, currVal, invoke, false, keys);
 
-                updateCache(tester, currentValue + 1, invoke, true, keys);
+                updateCache(tester, currVal + 1, invoke, true, keys);
 
                 invoke = !invoke;
-                currentValue++;
+                currVal++;
 
                 synchronized (lastUpdateTs) {
                     lastUpdateTs.set(System.currentTimeMillis());
@@ -180,14 +181,14 @@ public class IgniteCacheNearRestartRollbackSelfTest extends GridCommonAbstractTe
      * Updates the cache or rollback the update.
      *
      * @param ignite Ignite instance to use.
-     * @param newValue the new value to put to the entries
+     * @param newVal the new value to put to the entries
      * @param invoke whether to use invokeAll() or putAll()
      * @param rollback whether to rollback the changes or commit
      * @param keys Collection of keys to update.
      */
     private void updateCache(
         Ignite ignite,
-        int newValue,
+        int newVal,
         boolean invoke,
         boolean rollback,
         Set<Integer> keys
@@ -197,7 +198,7 @@ public class IgniteCacheNearRestartRollbackSelfTest extends GridCommonAbstractTe
         if (rollback) {
             while (true) {
                 try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                    updateEntries(cache, newValue, invoke, keys);
+                    updateEntries(cache, newVal, invoke, keys);
 
                     tx.rollback();
 
@@ -223,29 +224,30 @@ public class IgniteCacheNearRestartRollbackSelfTest extends GridCommonAbstractTe
             }
         }
         else
-            updateEntries(cache, newValue, invoke, keys);
+            updateEntries(cache, newVal, invoke, keys);
     }
 
     /**
      * Update the cache using either invokeAll() or putAll().
      *
      * @param cache the cache
-     * @param newValue the new value to put to the entries
+     * @param newVal the new value to put to the entries
      * @param invoke whether to use invokeAll() or putAll()
+     * @param keys Keys to update.
      */
     private void updateEntries(
         Cache<Integer, Integer> cache,
-        int newValue,
+        int newVal,
         boolean invoke,
         Set<Integer> keys
     ) {
         if (invoke)
-            cache.invokeAll(keys, new IntegerSetValue(newValue));
+            cache.invokeAll(keys, new IntegerSetValue(newVal));
         else {
             final Map<Integer, Integer> entries = new HashMap<>(ENTRY_COUNT);
 
             for (final Integer key : keys)
-                entries.put(key, newValue);
+                entries.put(key, newVal);
 
             cache.putAll(entries);
         }
@@ -256,19 +258,19 @@ public class IgniteCacheNearRestartRollbackSelfTest extends GridCommonAbstractTe
      */
     private static class IntegerSetValue implements EntryProcessor<Integer, Integer, Boolean>, Serializable {
         /** */
-        private final int newValue;
+        private final int newVal;
 
         /**
-         * @param newValue New value.
+         * @param newVal New value.
          */
-        private IntegerSetValue(final int newValue) {
-            this.newValue = newValue;
+        private IntegerSetValue(final int newVal) {
+            this.newVal = newVal;
         }
 
         /** {@inheritDoc} */
         @Override public Boolean process(MutableEntry<Integer, Integer> entry, Object... arguments)
             throws EntryProcessorException {
-            entry.setValue(newValue);
+            entry.setValue(newVal);
 
             return Boolean.TRUE;
         }


[07/21] ignite git commit: Fixed "IGNITE-2466 OutOfMemory when ONHEAP_TIERED mode is used".

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/core/src/test/java/org/apache/ignite/spi/communication/GridTestMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridTestMessage.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridTestMessage.java
index 368ca1e..808ff85 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridTestMessage.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridTestMessage.java
@@ -95,6 +95,11 @@ public class GridTestMessage implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessage.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessage.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessage.java
index a2833db..d528c47 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessage.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessage.java
@@ -39,6 +39,11 @@ public abstract class GridH2ValueMessage implements Message {
     public abstract Value value(GridKernalContext ctx) throws IgniteCheckedException;
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         return true;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/933d314e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index c027b26..dc572e2 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -574,6 +574,11 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         }
 
         /** {@inheritDoc} */
+        @Override public void onAckReceived() {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
         @Nullable @Override public <T> T value(CacheObjectContext ctx, boolean cpy) {
             return (T)val;
         }


[05/21] ignite git commit: Fixed CacheEvictableEntryImpl.equals()

Posted by sb...@apache.org.
Fixed CacheEvictableEntryImpl.equals()


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

Branch: refs/heads/ignite-1232
Commit: 7be1aab55799bf2807ec9957164ca636c469a7c3
Parents: cb35e1d
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Tue Feb 16 14:12:36 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Tue Feb 16 14:12:36 2016 -0800

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/CacheEvictableEntryImpl.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7be1aab5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java
index 9f4d9d7..be377c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java
@@ -221,7 +221,7 @@ public class CacheEvictableEntryImpl<K, V> implements EvictableEntry<K, V> {
         if (obj instanceof CacheEvictableEntryImpl) {
             CacheEvictableEntryImpl<K, V> other = (CacheEvictableEntryImpl<K, V>)obj;
 
-            return cached.key().equals(other.getKey());
+            return cached.key().equals(other.cached.key());
         }
 
         return false;


[09/21] ignite git commit: IGNITE-2652: CPP: Fixed a bug in QueryFieldsRow::GetNext() method. This closes #483.

Posted by sb...@apache.org.
IGNITE-2652: CPP: Fixed a bug in QueryFieldsRow::GetNext() method. This closes #483.


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

Branch: refs/heads/ignite-1232
Commit: 8562b001052e398287ef288b4226fa1cf07fcd0e
Parents: 933d314
Author: isapego <is...@gridgain.com>
Authored: Wed Feb 17 12:44:12 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Feb 17 12:44:12 2016 +0300

----------------------------------------------------------------------
 .../cpp/core-test/src/cache_query_test.cpp      | 45 ++++++++++++++++++++
 .../ignite/cache/query/query_fields_row.h       |  2 +-
 2 files changed, 46 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8562b001/modules/platforms/cpp/core-test/src/cache_query_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/cache_query_test.cpp b/modules/platforms/cpp/core-test/src/cache_query_test.cpp
index 18fe24f..05e6477 100644
--- a/modules/platforms/cpp/core-test/src/cache_query_test.cpp
+++ b/modules/platforms/cpp/core-test/src/cache_query_test.cpp
@@ -713,6 +713,51 @@ BOOST_AUTO_TEST_CASE(TestFieldsQuerySingle)
 }
 
 /**
+ * Test fields query with single entry.
+ */
+BOOST_AUTO_TEST_CASE(TestFieldsQueryExceptions)
+{
+    // Test simple query.
+    Cache<int, QueryPerson> cache = GetCache();
+
+    // Test query with two fields of different type.
+    SqlFieldsQuery qry("select age, name from QueryPerson");
+
+    QueryFieldsCursor cursor = cache.Query(qry);
+    CheckEmpty(cursor);
+
+    // Test simple query.
+    cache.Put(1, QueryPerson("A1", 10));
+
+    cursor = cache.Query(qry);
+
+    try
+    {
+        BOOST_REQUIRE(cursor.HasNext());
+
+        QueryFieldsRow row = cursor.GetNext();
+
+        BOOST_REQUIRE(row.HasNext());
+
+        int age = row.GetNext<int>();
+
+        BOOST_REQUIRE(age == 10);
+
+        std::string name = row.GetNext<std::string>();
+
+        BOOST_REQUIRE(name == "A1");
+
+        BOOST_REQUIRE(!row.HasNext());
+
+        CheckEmpty(cursor);
+    }
+    catch (IgniteError& error)
+    {
+        BOOST_FAIL(error.GetText());
+    }
+}
+
+/**
  * Test fields query with two simultaneously handled rows.
  */
 BOOST_AUTO_TEST_CASE(TestFieldsQueryTwo)

http://git-wip-us.apache.org/repos/asf/ignite/blob/8562b001/modules/platforms/cpp/core/include/ignite/cache/query/query_fields_row.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/query_fields_row.h b/modules/platforms/cpp/core/include/ignite/cache/query/query_fields_row.h
index bb10e9e..4f3be4c 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/query/query_fields_row.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/query/query_fields_row.h
@@ -109,7 +109,7 @@ namespace ignite
                 {
                     IgniteError err;
 
-                    QueryFieldsRow res = GetNext<T>(err);
+                    T res = GetNext<T>(err);
 
                     IgniteError::ThrowIfNeeded(err);
 


[16/21] ignite git commit: IGNITE-2678 Fix types in JNI interop for configuration-related methods

Posted by sb...@apache.org.
IGNITE-2678 Fix types in JNI interop for configuration-related methods


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

Branch: refs/heads/ignite-1232
Commit: d2e6535deeb217a33c1e79c782bb18d2eae520c8
Parents: a2f66b9
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Wed Feb 17 19:05:23 2016 +0300
Committer: Pavel Tupitsyn <pt...@gridgain.com>
Committed: Wed Feb 17 19:05:23 2016 +0300

----------------------------------------------------------------------
 .../cpp/common/include/ignite/common/exports.h          |  6 +++---
 .../platforms/cpp/common/include/ignite/common/java.h   | 12 ++++++------
 modules/platforms/cpp/common/src/exports.cpp            |  6 +++---
 modules/platforms/cpp/common/src/java.cpp               | 12 ++++++------
 4 files changed, 18 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d2e6535d/modules/platforms/cpp/common/include/ignite/common/exports.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/common/exports.h b/modules/platforms/cpp/common/include/ignite/common/exports.h
index 15911a6..6f2049d 100644
--- a/modules/platforms/cpp/common/include/ignite/common/exports.h
+++ b/modules/platforms/cpp/common/include/ignite/common/exports.h
@@ -36,8 +36,8 @@ extern "C" {
     void* IGNITE_CALL IgniteProcessorCache(gcj::JniContext* ctx, void* obj, char* name);
     void* IGNITE_CALL IgniteProcessorCreateCache(gcj::JniContext* ctx, void* obj, char* name);
     void* IGNITE_CALL IgniteProcessorGetOrCreateCache(gcj::JniContext* ctx, void* obj, char* name);
-    void* IGNITE_CALL IgniteProcessorCreateCacheFromConfig(gcj::JniContext* ctx, void* obj, long memPtr);
-    void* IGNITE_CALL IgniteProcessorGetOrCreateCacheFromConfig(gcj::JniContext* ctx, void* obj, long memPtr);
+    void* IGNITE_CALL IgniteProcessorCreateCacheFromConfig(gcj::JniContext* ctx, void* obj, long long memPtr);
+    void* IGNITE_CALL IgniteProcessorGetOrCreateCacheFromConfig(gcj::JniContext* ctx, void* obj, long long memPtr);
     void IGNITE_CALL IgniteProcessorDestroyCache(gcj::JniContext* ctx, void* obj, char* name);
     void* IGNITE_CALL IgniteProcessorAffinity(gcj::JniContext* ctx, void* obj, char* name);
     void* IGNITE_CALL IgniteProcessorDataStreamer(gcj::JniContext* ctx, void* obj, char* name, bool keepPortable);
@@ -50,7 +50,7 @@ extern "C" {
     void* IGNITE_CALL IgniteProcessorAtomicLong(gcj::JniContext* ctx, void* obj, char* name, long long initVal, bool create);
     void* IGNITE_CALL IgniteProcessorAtomicSequence(gcj::JniContext* ctx, void* obj, char* name, long long initVal, bool create);
     void* IGNITE_CALL IgniteProcessorAtomicReference(gcj::JniContext* ctx, void* obj, char* name, long long memPtr, bool create);
-    void IGNITE_CALL IgniteProcessorGetIgniteConfiguration(gcj::JniContext* ctx, void* obj, long memPtr);
+    void IGNITE_CALL IgniteProcessorGetIgniteConfiguration(gcj::JniContext* ctx, void* obj, long long memPtr);
     
     long long IGNITE_CALL IgniteTargetInStreamOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr);
     void IGNITE_CALL IgniteTargetInStreamOutStream(gcj::JniContext* ctx, void* obj, int opType, long long inMemPtr, long long outMemPtr);

http://git-wip-us.apache.org/repos/asf/ignite/blob/d2e6535d/modules/platforms/cpp/common/include/ignite/common/java.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/common/java.h b/modules/platforms/cpp/common/include/ignite/common/java.h
index ed47bc3..4b79665 100644
--- a/modules/platforms/cpp/common/include/ignite/common/java.h
+++ b/modules/platforms/cpp/common/include/ignite/common/java.h
@@ -515,10 +515,10 @@ namespace ignite
                 jobject ProcessorCreateCache(jobject obj, const char* name, JniErrorInfo* errInfo);
                 jobject ProcessorGetOrCreateCache(jobject obj, const char* name);
                 jobject ProcessorGetOrCreateCache(jobject obj, const char* name, JniErrorInfo* errInfo);
-                jobject ProcessorCreateCacheFromConfig(jobject obj, long memPtr);
-                jobject ProcessorCreateCacheFromConfig(jobject obj, long memPtr, JniErrorInfo* errInfo);
-                jobject ProcessorGetOrCreateCacheFromConfig(jobject obj, long memPtr);
-                jobject ProcessorGetOrCreateCacheFromConfig(jobject obj, long memPtr, JniErrorInfo* errInfo);
+                jobject ProcessorCreateCacheFromConfig(jobject obj, long long memPtr);
+                jobject ProcessorCreateCacheFromConfig(jobject obj, long long memPtr, JniErrorInfo* errInfo);
+                jobject ProcessorGetOrCreateCacheFromConfig(jobject obj, long long memPtr);
+                jobject ProcessorGetOrCreateCacheFromConfig(jobject obj, long long memPtr, JniErrorInfo* errInfo);
                 void ProcessorDestroyCache(jobject obj, const char* name);
                 void ProcessorDestroyCache(jobject obj, const char* name, JniErrorInfo* errInfo);
                 jobject ProcessorAffinity(jobject obj, const char* name);
@@ -532,7 +532,7 @@ namespace ignite
                 jobject ProcessorAtomicLong(jobject obj, char* name, long long initVal, bool create);
                 jobject ProcessorAtomicSequence(jobject obj, char* name, long long initVal, bool create);
                 jobject ProcessorAtomicReference(jobject obj, char* name, long long memPtr, bool create);
-				void ProcessorGetIgniteConfiguration(jobject obj, long memPtr);
+				void ProcessorGetIgniteConfiguration(jobject obj, long long memPtr);
                 
                 long long TargetInStreamOutLong(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL);
                 void TargetInStreamOutStream(jobject obj, int opType, long long inMemPtr, long long outMemPtr, JniErrorInfo* errInfo = NULL);
@@ -664,7 +664,7 @@ namespace ignite
                 void ExceptionCheck(JNIEnv* env, JniErrorInfo* errInfo);
                 jobject LocalToGlobal(JNIEnv* env, jobject obj);
                 jobject ProcessorCache0(jobject proc, const char* name, jmethodID mthd, JniErrorInfo* errInfo);
-                jobject ProcessorCacheFromConfig0(jobject proc, long memPtr, jmethodID mthd, JniErrorInfo* errInfo);
+                jobject ProcessorCacheFromConfig0(jobject proc, long long memPtr, jmethodID mthd, JniErrorInfo* errInfo);
             };
 
             JNIEXPORT jlong JNICALL JniCacheStoreCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr);

http://git-wip-us.apache.org/repos/asf/ignite/blob/d2e6535d/modules/platforms/cpp/common/src/exports.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/src/exports.cpp b/modules/platforms/cpp/common/src/exports.cpp
index fff2a16..93fd337 100644
--- a/modules/platforms/cpp/common/src/exports.cpp
+++ b/modules/platforms/cpp/common/src/exports.cpp
@@ -66,11 +66,11 @@ extern "C" {
         return ctx->ProcessorGetOrCreateCache(static_cast<jobject>(obj), name);
     }
 
-    void* IGNITE_CALL IgniteProcessorCreateCacheFromConfig(gcj::JniContext* ctx, void* obj, long memPtr) {
+    void* IGNITE_CALL IgniteProcessorCreateCacheFromConfig(gcj::JniContext* ctx, void* obj, long long memPtr) {
         return ctx->ProcessorCreateCacheFromConfig(static_cast<jobject>(obj), memPtr);
     }
 
-    void* IGNITE_CALL IgniteProcessorGetOrCreateCacheFromConfig(gcj::JniContext* ctx, void* obj, long memPtr) {
+    void* IGNITE_CALL IgniteProcessorGetOrCreateCacheFromConfig(gcj::JniContext* ctx, void* obj, long long memPtr) {
         return ctx->ProcessorGetOrCreateCacheFromConfig(static_cast<jobject>(obj), memPtr);
     }
 
@@ -122,7 +122,7 @@ extern "C" {
         return ctx->ProcessorAtomicReference(static_cast<jobject>(obj), name, memPtr, create);
     }
 
-	void IGNITE_CALL IgniteProcessorGetIgniteConfiguration(gcj::JniContext* ctx, void* obj, long memPtr) {
+	void IGNITE_CALL IgniteProcessorGetIgniteConfiguration(gcj::JniContext* ctx, void* obj, long long memPtr) {
         return ctx->ProcessorGetIgniteConfiguration(static_cast<jobject>(obj), memPtr);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d2e6535d/modules/platforms/cpp/common/src/java.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/src/java.cpp b/modules/platforms/cpp/common/src/java.cpp
index 8fc2293..89f4713 100644
--- a/modules/platforms/cpp/common/src/java.cpp
+++ b/modules/platforms/cpp/common/src/java.cpp
@@ -1166,7 +1166,7 @@ namespace ignite
                 return LocalToGlobal(env, cache);
             }
 
-            jobject JniContext::ProcessorCacheFromConfig0(jobject obj, long memPtr, jmethodID mthd, JniErrorInfo* errInfo)
+            jobject JniContext::ProcessorCacheFromConfig0(jobject obj, long long memPtr, jmethodID mthd, JniErrorInfo* errInfo)
             {
                 JNIEnv* env = Attach();
 
@@ -1221,20 +1221,20 @@ namespace ignite
                 ExceptionCheck(env, errInfo);
             }
 
-            jobject JniContext::ProcessorCreateCacheFromConfig(jobject obj, long memPtr) {
+            jobject JniContext::ProcessorCreateCacheFromConfig(jobject obj, long long memPtr) {
                 return ProcessorCreateCacheFromConfig(obj, memPtr, NULL);
             }
 
-            jobject JniContext::ProcessorCreateCacheFromConfig(jobject obj, long memPtr, JniErrorInfo* errInfo)
+            jobject JniContext::ProcessorCreateCacheFromConfig(jobject obj, long long memPtr, JniErrorInfo* errInfo)
             {
                 return ProcessorCacheFromConfig0(obj, memPtr, jvm->GetMembers().m_PlatformProcessor_createCacheFromConfig, errInfo);
             }
 
-            jobject JniContext::ProcessorGetOrCreateCacheFromConfig(jobject obj, long memPtr) {
+            jobject JniContext::ProcessorGetOrCreateCacheFromConfig(jobject obj, long long memPtr) {
                 return ProcessorGetOrCreateCacheFromConfig(obj, memPtr, NULL);
             }
 
-            jobject JniContext::ProcessorGetOrCreateCacheFromConfig(jobject obj, long memPtr, JniErrorInfo* errInfo)
+            jobject JniContext::ProcessorGetOrCreateCacheFromConfig(jobject obj, long long memPtr, JniErrorInfo* errInfo)
             {
                 return ProcessorCacheFromConfig0(obj, memPtr, jvm->GetMembers().m_PlatformProcessor_getOrCreateCacheFromConfig, errInfo);
             }
@@ -1379,7 +1379,7 @@ namespace ignite
                 return LocalToGlobal(env, res);
             }
 
-            void JniContext::ProcessorGetIgniteConfiguration(jobject obj, long memPtr)
+            void JniContext::ProcessorGetIgniteConfiguration(jobject obj, long long memPtr)
             {
                 JNIEnv* env = Attach();
 


[14/21] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/ignite-1232
Commit: 9cb175fb61a47db7201e891010179a5fdf08053f
Parents: acaeafb a67fbfa
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Feb 17 16:11:51 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Feb 17 16:11:51 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/binary/BinaryUtils.java     |   5 +-
 .../cache/CacheEntryPredicateContainsValue.java |   8 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |  10 +-
 .../cache/CacheEnumOperationsAbstractTest.java  | 307 +++++++++++++++++++
 .../CacheEnumOperationsSingleNodeTest.java      |  28 ++
 .../cache/CacheEnumOperationsTest.java          |  28 ++
 .../testsuites/IgniteCacheTestSuite2.java       |   4 +
 pom.xml                                         |   2 +-
 8 files changed, 380 insertions(+), 12 deletions(-)
----------------------------------------------------------------------



[15/21] ignite git commit: IGNITE-2641: Now "SELECT [alias].*" is possible. This closes #486.

Posted by sb...@apache.org.
IGNITE-2641: Now "SELECT [alias].*" is possible. This closes #486.


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

Branch: refs/heads/ignite-1232
Commit: a2f66b986fdb1876c1aa0ece51d1d44b3934d262
Parents: 9cb175f
Author: dkarachentsev <dk...@gridgain.com>
Authored: Wed Feb 17 16:47:44 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Feb 17 16:47:44 2016 +0300

----------------------------------------------------------------------
 .../processors/query/h2/IgniteH2Indexing.java   | 95 +++++++++++---------
 .../h2/GridIndexingSpiAbstractSelfTest.java     | 38 +++++++-
 2 files changed, 91 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a2f66b98/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index be72888..288c2b3 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -17,42 +17,6 @@
 
 package org.apache.ignite.internal.processors.query.h2;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.Field;
-import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
-import java.math.BigDecimal;
-import java.sql.Connection;
-import java.sql.Date;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.sql.Types;
-import java.text.MessageFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import javax.cache.Cache;
-import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
@@ -151,6 +115,43 @@ import org.h2.value.ValueUuid;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
+import javax.cache.Cache;
+import javax.cache.CacheException;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.math.BigDecimal;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.text.MessageFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_DEBUG_CONSOLE;
 import static org.apache.ignite.IgniteSystemProperties.getString;
 import static org.apache.ignite.internal.processors.query.GridQueryIndexType.FULLTEXT;
@@ -917,7 +918,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
     /**
      * Executes regular query.
-     * Note that SQL query can not refer to table alias, so use full table name instead.
      *
      * @param spaceName Space name.
      * @param qry Query.
@@ -1112,16 +1112,29 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         String from = " ";
 
         qry = qry.trim();
+
         String upper = qry.toUpperCase();
 
         if (upper.startsWith("SELECT")) {
             qry = qry.substring(6).trim();
 
-            if (!qry.startsWith("*"))
-                throw new IgniteCheckedException("Only queries starting with 'SELECT *' are supported or " +
-                    "use SqlFieldsQuery instead: " + qry0);
+            final int star = qry.indexOf('*');
+
+            if (star == 0)
+                qry = qry.substring(1).trim();
+            else if (star > 0) {
+                if (F.eq('.', qry.charAt(star - 1))) {
+                    t = qry.substring(0, star - 1);
+
+                    qry = qry.substring(star + 1).trim();
+                }
+                else
+                    throw new IgniteCheckedException("Invalid query (missing alias before asterisk): " + qry0);
+            }
+            else
+                throw new IgniteCheckedException("Only queries starting with 'SELECT *' and 'SELECT alias.*' " +
+                    "are supported (rewrite your query or use SqlFieldsQuery instead): " + qry0);
 
-            qry = qry.substring(1).trim();
             upper = qry.toUpperCase();
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a2f66b98/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index dc572e2..0da71c8 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -25,7 +25,6 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.Callable;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -227,6 +226,25 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         assertFalse(spi.query(typeAB.space(), "select * from A.B", Collections.emptySet(), typeAB, null).hasNext());
         assertFalse(spi.query(typeBA.space(), "select * from B.A", Collections.emptySet(), typeBA, null).hasNext());
 
+        assertFalse(spi.query(typeBA.space(), "select * from B.A, A.B, A.A",
+            Collections.emptySet(), typeBA, null).hasNext());
+
+        try {
+            spi.query(typeBA.space(), "select aa.*, ab.*, ba.* from A.A aa, A.B ab, B.A ba",
+                Collections.emptySet(), typeBA, null).hasNext();
+
+            fail("Enumerations of aliases in select block must be prohibited");
+        }
+        catch (IgniteCheckedException e) {
+            // all fine
+        }
+
+        assertFalse(spi.query(typeAB.space(), "select ab.* from A.B ab",
+            Collections.emptySet(), typeAB, null).hasNext());
+
+        assertFalse(spi.query(typeBA.space(), "select   ba.*   from B.A  as ba",
+            Collections.emptySet(), typeBA, null).hasNext());
+
         // Nothing to remove.
         spi.remove("A", key(1), aa(1, "", 10));
         spi.remove("B", key(1), ba(1, "", 10, true));
@@ -287,6 +305,15 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         assertEquals(aa(2, "Valera", 19).value(null, false), value(res.next()));
         assertFalse(res.hasNext());
 
+        res = spi.query(typeAA.space(), "select aa.* from a aa order by aa.age",
+            Collections.emptySet(), typeAA, null);
+
+        assertTrue(res.hasNext());
+        assertEquals(aa(3, "Borya", 18).value(null, false), value(res.next()));
+        assertTrue(res.hasNext());
+        assertEquals(aa(2, "Valera", 19).value(null, false), value(res.next()));
+        assertFalse(res.hasNext());
+
         res = spi.query(typeAB.space(), "from b order by name", Collections.emptySet(), typeAB, null);
 
         assertTrue(res.hasNext());
@@ -295,6 +322,15 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         assertEquals(ab(4, "Vitalya", 20, "Very Good guy").value(null, false), value(res.next()));
         assertFalse(res.hasNext());
 
+        res = spi.query(typeAB.space(), "select bb.* from b as bb order by bb.name",
+            Collections.emptySet(), typeAB, null);
+
+        assertTrue(res.hasNext());
+        assertEquals(ab(1, "Vasya", 20, "Some text about Vasya goes here.").value(null, false), value(res.next()));
+        assertTrue(res.hasNext());
+        assertEquals(ab(4, "Vitalya", 20, "Very Good guy").value(null, false), value(res.next()));
+        assertFalse(res.hasNext());
+
         res = spi.query(typeBA.space(), "from a", Collections.emptySet(), typeBA, null);
 
         assertTrue(res.hasNext());


[21/21] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-1232

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/master' into ignite-1232

Conflicts:
	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
	modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java


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

Branch: refs/heads/ignite-1232
Commit: fce714c0651b4d39671dda91bf7114d57120d35d
Parents: 97cd783
Author: sboikov <sb...@gridgain.com>
Authored: Thu Feb 18 09:04:03 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Feb 18 09:04:03 2016 +0300

----------------------------------------------------------------------
 .../query/h2/twostep/msg/GridH2IndexRangeRequest.java        | 5 +++++
 .../query/h2/twostep/msg/GridH2IndexRangeResponse.java       | 5 +++++
 .../processors/query/h2/twostep/msg/GridH2QueryRequest.java  | 5 +++++
 .../processors/query/h2/twostep/msg/GridH2RowMessage.java    | 5 +++++
 .../processors/query/h2/twostep/msg/GridH2RowRange.java      | 5 +++++
 .../query/h2/twostep/msg/GridH2RowRangeBounds.java           | 5 +++++
 .../processors/query/h2/GridIndexingSpiAbstractSelfTest.java | 8 ++++----
 7 files changed, 34 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fce714c0/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java
index 2257c4c..075f50d28 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java
@@ -197,6 +197,11 @@ public class GridH2IndexRangeRequest implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridH2IndexRangeRequest.class, this, "boundsSize", bounds == null ? null : bounds.size());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fce714c0/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeResponse.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeResponse.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeResponse.java
index f75707f..f557a35 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeResponse.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeResponse.java
@@ -268,6 +268,11 @@ public class GridH2IndexRangeResponse implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridH2IndexRangeResponse.class, this, "rangesSize", ranges == null ? null : ranges.size());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fce714c0/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java
index bd932f9..921d101 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java
@@ -389,6 +389,11 @@ public class GridH2QueryRequest implements Message, GridCacheQueryMarshallable {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridH2QueryRequest.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fce714c0/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowMessage.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowMessage.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowMessage.java
index ca2c3bb..2100847 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowMessage.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowMessage.java
@@ -105,6 +105,11 @@ public class GridH2RowMessage implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridH2RowMessage.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fce714c0/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowRange.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowRange.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowRange.java
index 55ee5e3..b38a44d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowRange.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowRange.java
@@ -165,4 +165,9 @@ public class GridH2RowRange implements Message {
     @Override public byte fieldsCount() {
         return 3;
     }
+
+    /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fce714c0/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowRangeBounds.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowRangeBounds.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowRangeBounds.java
index f5f70a3..2dd1e34 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowRangeBounds.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowRangeBounds.java
@@ -177,6 +177,11 @@ public class GridH2RowRangeBounds implements Message {
     }
 
     /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridH2RowRangeBounds.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fce714c0/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index bb96cd8..fae3188 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -226,11 +226,11 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         assertFalse(spi.queryLocalSql(typeAB.space(), "select * from A.B", Collections.emptySet(), typeAB, null).hasNext());
         assertFalse(spi.queryLocalSql(typeBA.space(), "select * from B.A", Collections.emptySet(), typeBA, null).hasNext());
 
-        assertFalse(spi.query(typeBA.space(), "select * from B.A, A.B, A.A",
+        assertFalse(spi.queryLocalSql(typeBA.space(), "select * from B.A, A.B, A.A",
             Collections.emptySet(), typeBA, null).hasNext());
 
         try {
-            spi.query(typeBA.space(), "select aa.*, ab.*, ba.* from A.A aa, A.B ab, B.A ba",
+            spi.queryLocalSql(typeBA.space(), "select aa.*, ab.*, ba.* from A.A aa, A.B ab, B.A ba",
                 Collections.emptySet(), typeBA, null).hasNext();
 
             fail("Enumerations of aliases in select block must be prohibited");
@@ -239,10 +239,10 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
             // all fine
         }
 
-        assertFalse(spi.query(typeAB.space(), "select ab.* from A.B ab",
+        assertFalse(spi.queryLocalSql(typeAB.space(), "select ab.* from A.B ab",
             Collections.emptySet(), typeAB, null).hasNext());
 
-        assertFalse(spi.query(typeBA.space(), "select   ba.*   from B.A  as ba",
+        assertFalse(spi.queryLocalSql(typeBA.space(), "select   ba.*   from B.A  as ba",
             Collections.emptySet(), typeBA, null).hasNext());
 
         // Nothing to remove.


[12/21] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/ignite-1232
Commit: a67fbfacac8f61a69732723b26b155cf64aadcaf
Parents: f0f9ef6 8be62f0
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Feb 17 15:08:16 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Feb 17 15:08:16 2016 +0300

----------------------------------------------------------------------
 .../cache/query/CacheQueryEntryEvent.java       |  48 +
 .../ignite/internal/GridJobCancelRequest.java   |   5 +
 .../ignite/internal/GridJobExecuteRequest.java  |   5 +
 .../ignite/internal/GridJobExecuteResponse.java |   5 +
 .../ignite/internal/GridJobSiblingsRequest.java |   5 +
 .../internal/GridJobSiblingsResponse.java       |   5 +
 .../ignite/internal/GridTaskCancelRequest.java  |   5 +
 .../ignite/internal/GridTaskSessionRequest.java |   5 +
 .../internal/binary/BinaryEnumObjectImpl.java   |   5 +
 .../internal/binary/BinaryObjectImpl.java       |   5 +
 .../binary/BinaryObjectOffheapImpl.java         |   5 +
 .../ignite/internal/binary/BinaryUtils.java     |   5 +-
 .../checkpoint/GridCheckpointRequest.java       |   5 +
 .../managers/communication/GridIoMessage.java   |   5 +
 .../communication/GridIoUserMessage.java        |   5 +
 .../deployment/GridDeploymentInfoBean.java      |   5 +
 .../deployment/GridDeploymentRequest.java       |   5 +
 .../deployment/GridDeploymentResponse.java      |   5 +
 .../eventstorage/GridEventStorageMessage.java   |   5 +
 .../affinity/AffinityTopologyVersion.java       |   5 +
 .../cache/CacheEntryInfoCollection.java         |   5 +
 .../cache/CacheEntryPredicateAdapter.java       |   5 +
 .../cache/CacheEntryPredicateContainsValue.java |   8 +-
 .../cache/CacheEntrySerializablePredicate.java  |   5 +
 .../cache/CacheEvictableEntryImpl.java          |   2 +-
 .../processors/cache/CacheEvictionEntry.java    |   5 +
 .../cache/CacheInvokeDirectResult.java          |   5 +
 .../cache/CacheObjectByteArrayImpl.java         |   5 +
 .../processors/cache/CacheObjectImpl.java       |   5 +
 .../processors/cache/GridCacheEntryInfo.java    |   5 +
 .../processors/cache/GridCacheMessage.java      |   5 +
 .../processors/cache/GridCacheReturn.java       |   5 +
 .../processors/cache/KeyCacheObjectImpl.java    |   5 +
 .../binary/CacheObjectBinaryProcessorImpl.java  |  10 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   5 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |  14 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |  16 +
 .../preloader/GridDhtPartitionExchangeId.java   |   5 +
 .../distributed/near/CacheVersionedValue.java   |   5 +
 .../cache/query/GridCacheSqlQuery.java          |   5 +
 .../continuous/CacheContinuousQueryEntry.java   |   5 +
 .../continuous/CacheContinuousQueryEvent.java   |  17 +-
 .../continuous/CacheContinuousQueryHandler.java |  95 +-
 .../CacheContinuousQueryListener.java           |   3 +-
 .../continuous/CacheContinuousQueryManager.java |  26 +-
 .../cache/transactions/IgniteTxEntry.java       |   5 +
 .../cache/transactions/IgniteTxKey.java         |   5 +
 .../cache/transactions/TxEntryValueHolder.java  |   5 +
 .../cache/version/GridCacheVersion.java         |   5 +
 .../clock/GridClockDeltaSnapshotMessage.java    |   5 +
 .../processors/clock/GridClockDeltaVersion.java |   5 +
 .../continuous/GridContinuousMessage.java       |   5 +
 .../datastreamer/DataStreamerEntry.java         |   5 +
 .../datastreamer/DataStreamerRequest.java       |   5 +
 .../datastreamer/DataStreamerResponse.java      |   5 +
 .../processors/igfs/IgfsAckMessage.java         |   5 +
 .../internal/processors/igfs/IgfsBlockKey.java  |   5 +
 .../processors/igfs/IgfsBlocksMessage.java      |   5 +
 .../processors/igfs/IgfsDeleteMessage.java      |   5 +
 .../processors/igfs/IgfsFileAffinityRange.java  |   5 +
 .../igfs/IgfsFragmentizerRequest.java           |   5 +
 .../igfs/IgfsFragmentizerResponse.java          |   5 +
 .../processors/igfs/IgfsSyncMessage.java        |   5 +
 .../messages/GridQueryCancelRequest.java        |   5 +
 .../twostep/messages/GridQueryFailResponse.java |   5 +
 .../messages/GridQueryNextPageRequest.java      |   5 +
 .../messages/GridQueryNextPageResponse.java     |   5 +
 .../h2/twostep/messages/GridQueryRequest.java   |   5 +
 .../handlers/task/GridTaskResultRequest.java    |   5 +
 .../handlers/task/GridTaskResultResponse.java   |   5 +
 .../ignite/internal/util/GridByteArrayList.java |   5 +
 .../ignite/internal/util/GridLongList.java      |   5 +
 .../internal/util/GridMessageCollection.java    |   5 +
 .../internal/util/UUIDCollectionMessage.java    |   5 +
 .../util/nio/GridNioFinishedFuture.java         |   5 +
 .../ignite/internal/util/nio/GridNioFuture.java |   7 +-
 .../internal/util/nio/GridNioFutureImpl.java    |   5 +
 .../util/nio/GridNioRecoveryDescriptor.java     |   2 +
 .../ignite/internal/util/nio/GridNioServer.java |   7 +
 .../extensions/communication/Message.java       |   5 +
 .../jobstealing/JobStealingRequest.java         |   5 +
 .../communication/tcp/TcpCommunicationSpi.java  |  15 +
 .../internal/GridAffinityNoCacheSelfTest.java   |   5 +
 .../GridCommunicationSendMessageSelfTest.java   |   5 +
 .../communication/GridIoManagerSelfTest.java    |   5 +
 .../cache/CacheEnumOperationsAbstractTest.java  | 307 +++++++
 .../CacheEnumOperationsSingleNodeTest.java      |  28 +
 .../cache/CacheEnumOperationsTest.java          |  28 +
 .../GridCacheEvictableEntryEqualsSelfTest.java  |  85 ++
 ...CacheContinuousQueryCounterAbstractTest.java | 612 +++++++++++++
 ...inuousQueryCounterPartitionedAtomicTest.java |  41 +
 ...ContinuousQueryCounterPartitionedTxTest.java |  41 +
 ...tinuousQueryCounterReplicatedAtomicTest.java |  41 +
 ...eContinuousQueryCounterReplicatedTxTest.java |  41 +
 ...acheContinuousQueryRandomOperationsTest.java | 896 ++++++++++++++++---
 .../communication/GridTestMessage.java          |   5 +
 .../spi/communication/GridTestMessage.java      |   5 +
 .../IgniteCacheEvictionSelfTestSuite.java       |   4 +-
 .../testsuites/IgniteCacheTestSuite2.java       |   4 +
 .../h2/twostep/msg/GridH2ValueMessage.java      |   5 +
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   5 +
 .../IgniteCacheQuerySelfTestSuite.java          |   8 +
 .../cpp/core-test/src/cache_query_test.cpp      |  45 +
 .../ignite/cache/query/query_fields_row.h       |   2 +-
 104 files changed, 2588 insertions(+), 240 deletions(-)
----------------------------------------------------------------------



[19/21] ignite git commit: ignite-2669 Register BinaryEnumObjectImpl as predefined type.

Posted by sb...@apache.org.
ignite-2669 Register BinaryEnumObjectImpl as predefined type.


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

Branch: refs/heads/ignite-1232
Commit: 21f22d3db84a731de54db9530a04244321766c57
Parents: 7d5f77e
Author: sboikov <sb...@gridgain.com>
Authored: Thu Feb 18 08:06:32 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Feb 18 08:06:32 2016 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/internal/binary/BinaryContext.java  | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/21f22d3d/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
index cf19bdf..4df9ba2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
@@ -218,6 +218,7 @@ public class BinaryContext {
         registerPredefinedType(BinaryObjectOffheapImpl.class, 0);
         registerPredefinedType(BinaryMetadataKey.class, 0);
         registerPredefinedType(BinaryMetadata.class, 0);
+        registerPredefinedType(BinaryEnumObjectImpl.class, 0);
 
         // IDs range [200..1000] is used by Ignite internal APIs.
     }