You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2016/12/21 11:32:00 UTC

[01/16] ignite git commit: ignite-4371 Avoid synchronous 'rollback' call from system threads

Repository: ignite
Updated Branches:
  refs/heads/ignite-2.0 b2cc951e3 -> 0a3364453


ignite-4371 Avoid synchronous 'rollback' call from system threads


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

Branch: refs/heads/ignite-2.0
Commit: 0c782b0b6c210e837ed37ffb9b1eb5cdb7db5662
Parents: b2cc951
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 16 19:15:48 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 16 19:15:48 2016 +0300

----------------------------------------------------------------------
 .../GridCachePartitionExchangeManager.java      |  37 +++---
 .../processors/cache/GridCacheProcessor.java    |   2 +-
 .../processors/cache/GridCacheUtils.java        |  10 +-
 .../GridDistributedTxRemoteAdapter.java         |   2 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |  74 +++++------
 .../cache/distributed/dht/GridDhtTxLocal.java   | 125 +++++++------------
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   7 +-
 .../near/GridNearTxFinishFuture.java            |  44 +++----
 .../cache/distributed/near/GridNearTxLocal.java |  48 ++++---
 .../cache/transactions/IgniteTxHandler.java     |  13 +-
 .../transactions/IgniteTxLocalAdapter.java      |   3 +-
 .../cache/transactions/IgniteTxManager.java     |   7 +-
 .../GridCacheMissingCommitVersionSelfTest.java  |   6 +-
 .../IgniteTxStoreExceptionAbstractSelfTest.java |   1 +
 14 files changed, 161 insertions(+), 218 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0c782b0b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 8ea2169..ef8c994e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -57,6 +57,7 @@ import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.events.DiscoveryCustomEvent;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
@@ -852,24 +853,30 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                     ready = cacheCtx.started();
 
                 if (ready) {
-                    GridDhtPartitionFullMap locMap = cacheCtx.topology().partitionMap(true);
+                    GridAffinityAssignmentCache affCache = cacheCtx.affinity().affinityCache();
 
-                    if (useOldApi) {
-                        locMap = new GridDhtPartitionFullMap(locMap.nodeId(),
-                            locMap.nodeOrder(),
-                            locMap.updateSequence(),
-                            locMap);
-                    }
+                    if (affCache != null) {
+                        GridDhtPartitionFullMap locMap = cacheCtx.topology().partitionMap(true);
+
+                        if (useOldApi) {
+                            locMap = new GridDhtPartitionFullMap(locMap.nodeId(),
+                                locMap.nodeOrder(),
+                                locMap.updateSequence(),
+                                locMap);
+                        }
 
-                    addFullPartitionsMap(m,
-                        dupData,
-                        compress,
-                        cacheCtx.cacheId(),
-                        locMap,
-                        cacheCtx.affinity().affinityCache().similarAffinityKey());
+                        addFullPartitionsMap(m,
+                            dupData,
+                            compress,
+                            cacheCtx.cacheId(),
+                            locMap,
+                            affCache.similarAffinityKey());
 
-                    if (exchId != null)
-                        m.addPartitionUpdateCounters(cacheCtx.cacheId(), cacheCtx.topology().updateCounters(true));
+                        if (exchId != null)
+                            m.addPartitionUpdateCounters(cacheCtx.cacheId(), cacheCtx.topology().updateCounters(true));
+                    }
+                    else
+                        assert cctx.cacheContext(cacheCtx.cacheId()) == null : cacheCtx.name();
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c782b0b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 0be2072..9487589 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -1761,7 +1761,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /**
      * @param req Stop request.
      */
-    public void prepareCacheStop(DynamicCacheChangeRequest req) {
+    private void prepareCacheStop(DynamicCacheChangeRequest req) {
         assert req.stop() || req.close() : req;
 
         GridCacheAdapter<?, ?> cache = caches.remove(maskNull(req.cacheName()));

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c782b0b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 47abf2f..969c41a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -911,9 +911,13 @@ public class GridCacheUtils {
         if (tx == null)
             return "null";
 
-        return tx.getClass().getSimpleName() + "[id=" + tx.xid() + ", concurrency=" + tx.concurrency() +
-            ", isolation=" + tx.isolation() + ", state=" + tx.state() + ", invalidate=" + tx.isInvalidate() +
-            ", rollbackOnly=" + tx.isRollbackOnly() + ", nodeId=" + tx.nodeId() +
+        return tx.getClass().getSimpleName() + "[id=" + tx.xid() +
+            ", concurrency=" + tx.concurrency() +
+            ", isolation=" + tx.isolation() +
+            ", state=" + tx.state() +
+            ", invalidate=" + tx.isInvalidate() +
+            ", rollbackOnly=" + tx.isRollbackOnly() +
+            ", nodeId=" + tx.nodeId() +
             ", duration=" + (U.currentTimeMillis() - tx.startTime()) + ']';
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c782b0b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
index 4adfa8b..68c0e57 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
@@ -386,7 +386,7 @@ public class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
         // If another thread is doing prepare or rollback.
         if (!state(PREPARING)) {
             // In optimistic mode prepare may be called multiple times.
-            if(state() != PREPARING || !optimistic()) {
+            if (state() != PREPARING || !optimistic()) {
                 if (log.isDebugEnabled())
                     log.debug("Invalid transaction state for prepare: " + this);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c782b0b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
index ac2ab41..147cbea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
@@ -33,8 +33,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxMapping;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
-import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException;
-import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.util.future.GridCompoundIdentityFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -94,9 +92,6 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
     /** Near mappings. */
     private Map<UUID, GridDistributedTxMapping> nearMap;
 
-    /** Trackable flag. */
-    private boolean trackable = true;
-
     /**
      * @param cctx Context.
      * @param tx Transaction.
@@ -151,46 +146,24 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
 
     /** {@inheritDoc} */
     @Override public boolean trackable() {
-        return trackable;
+        return true;
     }
 
     /** {@inheritDoc} */
     @Override public void markNotTrackable() {
-        trackable = false;
+        assert false;
     }
 
     /**
      * @param e Error.
      */
-    public void onError(Throwable e) {
-        if (ERR_UPD.compareAndSet(this, null, e)) {
-            boolean marked = tx.setRollbackOnly();
-
-            if (e instanceof IgniteTxRollbackCheckedException) {
-                if (marked) {
-                    try {
-                        tx.rollback();
-                    }
-                    catch (IgniteCheckedException ex) {
-                        U.error(log, "Failed to automatically rollback transaction: " + tx, ex);
-                    }
-                }
-            }
-            else if (tx.isSystemInvalidate()) { // Invalidate remote transactions on heuristic error.
-                finish();
+    public void rollbackOnError(Throwable e) {
+        assert e != null;
 
-                try {
-                    get();
-                }
-                catch (IgniteTxHeuristicCheckedException ignore) {
-                    // Future should complete with GridCacheTxHeuristicException.
-                }
-                catch (IgniteCheckedException err) {
-                    U.error(log, "Failed to invalidate transaction: " + tx, err);
-                }
-            }
+        if (ERR_UPD.compareAndSet(this, null, e)) {
+            tx.setRollbackOnly();
 
-            onComplete();
+            finish(false);
         }
     }
 
@@ -240,12 +213,21 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
     /** {@inheritDoc} */
     @Override public boolean onDone(IgniteInternalTx tx, Throwable err) {
         if (initialized() || err != null) {
-            if (this.tx.onePhaseCommit() && (this.tx.state() == COMMITTING))
-                this.tx.tmFinish(err == null);
-
             Throwable e = this.err;
 
-            if (e == null && commit)
+            if (this.tx.onePhaseCommit() && (this.tx.state() == COMMITTING)) {
+                try {
+                    this.tx.tmFinish(err == null);
+                }
+                catch (IgniteCheckedException finishErr) {
+                    U.error(log, "Failed to finish tx: " + tx, e);
+
+                    if (e == null)
+                        e = finishErr;
+                }
+            }
+
+            if (commit && e == null)
                 e = this.tx.commitError();
 
             Throwable finishErr = e != null ? e : err;
@@ -255,7 +237,7 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
                     finishErr = this.tx.commitError();
 
                 if (this.tx.syncMode() != PRIMARY_SYNC)
-                    this.tx.sendFinishReply(commit, finishErr);
+                    this.tx.sendFinishReply(finishErr);
 
                 // Don't forget to clean up.
                 cctx.mvcc().removeFuture(futId);
@@ -284,13 +266,15 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
 
     /**
      * Initializes future.
+     *
+     * @param commit Commit flag.
      */
     @SuppressWarnings({"SimplifiableIfStatement", "IfMayBeConditional"})
-    public void finish() {
+    public void finish(boolean commit) {
         boolean sync;
 
         if (!F.isEmpty(dhtMap) || !F.isEmpty(nearMap))
-            sync = finish(dhtMap, nearMap);
+            sync = finish(commit, dhtMap, nearMap);
         else if (!commit && !F.isEmpty(tx.lockTransactionNodes()))
             sync = rollbackLockTransactions(tx.lockTransactionNodes());
         else
@@ -308,7 +292,6 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
      * @return {@code True} in case there is at least one synchronous {@code MiniFuture} to wait for.
      */
     private boolean rollbackLockTransactions(Collection<ClusterNode> nodes) {
-        assert !commit;
         assert !F.isEmpty(nodes);
 
         if (tx.onePhaseCommit())
@@ -337,7 +320,7 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
                 tx.commitVersion(),
                 tx.threadId(),
                 tx.isolation(),
-                commit,
+                false,
                 tx.isInvalidate(),
                 tx.system(),
                 tx.ioPolicy(),
@@ -390,11 +373,14 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
     }
 
     /**
+     * @param commit Commit flag.
      * @param dhtMap DHT map.
      * @param nearMap Near map.
      * @return {@code True} in case there is at least one synchronous {@code MiniFuture} to wait for.
      */
-    private boolean finish(Map<UUID, GridDistributedTxMapping> dhtMap, Map<UUID, GridDistributedTxMapping> nearMap) {
+    private boolean finish(boolean commit,
+        Map<UUID, GridDistributedTxMapping> dhtMap,
+        Map<UUID, GridDistributedTxMapping> nearMap) {
         if (tx.onePhaseCommit())
             return false;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c782b0b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
index b659abb..4e39e9b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
@@ -503,52 +503,57 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
     }
 
     /**
+     * @param commit Commit flag.
      * @param prepFut Prepare future.
      * @param fut Finish future.
      */
-    private void finishCommit(@Nullable IgniteInternalFuture prepFut, GridDhtTxFinishFuture fut) {
+    private void finishTx(boolean commit, @Nullable IgniteInternalFuture prepFut, GridDhtTxFinishFuture fut) {
+        assert prepFut == null || prepFut.isDone();
+
         boolean primarySync = syncMode() == PRIMARY_SYNC;
 
         IgniteCheckedException err = null;
 
-        try {
-            if (prepFut != null)
-                prepFut.get(); // Check for errors.
-
-            if (finish(true)) {
-                if (primarySync)
-                    sendFinishReply(true, null);
-
-                fut.finish();
+        if (!commit && prepFut != null) {
+            try {
+                prepFut.get();
             }
-            else {
-                err = new IgniteCheckedException("Failed to commit transaction: " + CU.txString(this));
-
-                fut.onError(err);
+            catch (IgniteCheckedException e) {
+                if (log.isDebugEnabled())
+                    log.debug("Failed to prepare transaction [tx=" + this + ", e=" + e + ']');
+            }
+            finally {
+                prepFut = null;
             }
         }
-        catch (IgniteTxOptimisticCheckedException e) {
-            if (log.isDebugEnabled())
-                log.debug("Failed to optimistically prepare transaction [tx=" + this + ", e=" + e + ']');
 
-            err = e;
+        try {
+            if (prepFut != null)
+                prepFut.get(); // Check for errors.
 
-            fut.onError(e);
+            boolean finished = finish(commit);
+
+            if (!finished)
+                err = new IgniteCheckedException("Failed to finish transaction [commit=" + commit +
+                    ", tx=" + CU.txString(this) + ']');
         }
         catch (IgniteCheckedException e) {
-            U.error(log, "Failed to prepare transaction: " + this, e);
+            U.error(log, "Failed to finish transaction [commit=" + commit + ", tx=" + this + ']', e);
 
             err = e;
-
-            fut.onError(e);
         }
 
-        if (primarySync && err != null)
-            sendFinishReply(true, err);
+        if (primarySync)
+            sendFinishReply(err);
+
+        if (err != null)
+            fut.rollbackOnError(err);
+        else
+            fut.finish(commit);
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings({"ThrowableInstanceNeverThrown"})
+    @SuppressWarnings("unchecked")
     @Override public IgniteInternalFuture<IgniteInternalTx> commitAsync() {
         if (log.isDebugEnabled())
             log.debug("Committing dht local tx: " + this);
@@ -557,7 +562,7 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
         if (pessimistic())
             prepareAsync();
 
-        final GridDhtTxFinishFuture fut = new GridDhtTxFinishFuture<>(cctx, this, /*commit*/true);
+        final GridDhtTxFinishFuture fut = new GridDhtTxFinishFuture<>(cctx, this, true);
 
         cctx.mvcc().addFuture(fut, fut.futureId());
 
@@ -565,11 +570,11 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
 
         if (prep != null) {
             if (prep.isDone())
-                finishCommit(prep, fut);
+                finishTx(true, prep, fut);
             else {
                 prep.listen(new CI1<IgniteInternalFuture<?>>() {
                     @Override public void apply(IgniteInternalFuture<?> f) {
-                        finishCommit(f, fut);
+                        finishTx(true, f, fut);
                     }
                 });
             }
@@ -577,7 +582,7 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
         else {
             assert optimistic();
 
-            finishCommit(null, fut);
+            finishTx(true, null, fut);
         }
 
         return fut;
@@ -590,70 +595,26 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
         PREP_FUT_UPD.compareAndSet(this, fut, null);
     }
 
-    /**
-     * @param prepFut Prepare future.
-     * @param fut Finish future.
-     */
-    private void finishRollback(@Nullable IgniteInternalFuture prepFut, GridDhtTxFinishFuture fut) {
-        try {
-            if (prepFut != null)
-                prepFut.get();
-        }
-        catch (IgniteCheckedException e) {
-            if (log.isDebugEnabled())
-                log.debug("Failed to prepare or rollback transaction [tx=" + this + ", e=" + e + ']');
-        }
-
-        boolean primarySync = syncMode() == PRIMARY_SYNC;
-
-        IgniteCheckedException err = null;
-
-        try {
-            if (finish(false) || state() == UNKNOWN) {
-                if (primarySync)
-                    sendFinishReply(false, null);
-
-                fut.finish();
-            }
-            else {
-                err = new IgniteCheckedException("Failed to rollback transaction: " +
-                    CU.txString(GridDhtTxLocal.this));
-
-                fut.onError(err);
-            }
-        }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to gracefully rollback transaction: " + CU.txString(GridDhtTxLocal.this),
-                e);
-
-            err = e;
-
-            fut.onError(e);
-        }
-
-        if (primarySync && err != null)
-            sendFinishReply(false, err);
-    }
-
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override public IgniteInternalFuture<IgniteInternalTx> rollbackAsync() {
-        GridDhtTxPrepareFuture prepFut = this.prepFut;
-
-        final GridDhtTxFinishFuture fut = new GridDhtTxFinishFuture<>(cctx, this, /*rollback*/false);
+        final GridDhtTxFinishFuture fut = new GridDhtTxFinishFuture<>(cctx, this, false);
 
         cctx.mvcc().addFuture(fut, fut.futureId());
 
+        GridDhtTxPrepareFuture prepFut = this.prepFut;
+
         if (prepFut != null) {
             prepFut.complete();
 
             prepFut.listen(new CI1<IgniteInternalFuture<?>>() {
                 @Override public void apply(IgniteInternalFuture<?> f) {
-                    finishRollback(f, fut);
+                    finishTx(false, f, fut);
                 }
             });
         }
         else
-            finishRollback(null, fut);
+            finishTx(false, null, fut);
 
         return fut;
     }
@@ -672,7 +633,7 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
     }
 
     /** {@inheritDoc} */
-    @Override protected void sendFinishReply(boolean commit, @Nullable Throwable err) {
+    @Override protected void sendFinishReply(@Nullable Throwable err) {
         if (nearFinFutId != null) {
             if (nearNodeId.equals(cctx.localNodeId())) {
                 if (log.isDebugEnabled())
@@ -701,8 +662,8 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
                 }
             }
             catch (Throwable ex) {
-                U.error(log, "Failed to send finish response to node (transaction was " +
-                    (commit ? "committed" : "rolledback") + ") [txId=" + nearXidVersion() +
+                U.error(log, "Failed to send finish response to node [txId=" + nearXidVersion() +
+                    ", txState=" + state() +
                     ", dhtTxId=" + xidVersion() +
                     ", node=" + nearNodeId +
                     ", res=" + res + ']', ex);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c782b0b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
index 35dfb62..1d88d84 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
@@ -236,10 +236,9 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
         AffinityTopologyVersion topVer);
 
     /**
-     * @param commit Commit flag.
      * @param err Error, if any.
      */
-    protected abstract void sendFinishReply(boolean commit, @Nullable Throwable err);
+    protected abstract void sendFinishReply(@Nullable Throwable err);
 
     /** {@inheritDoc} */
     @Override public boolean needsCompletedVersions() {
@@ -249,7 +248,7 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
     /**
      * @return Versions for all pending locks that were in queue before tx locks were released.
      */
-    public Collection<GridCacheVersion> pendingVersions() {
+    Collection<GridCacheVersion> pendingVersions() {
         return pendingVers == null ? Collections.<GridCacheVersion>emptyList() : pendingVers;
     }
 
@@ -726,7 +725,7 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
                         /*read*/read,
                         accessTtl,
                         filter == null ? CU.empty0() : filter,
-                        /**computeInvoke*/false);
+                        /*computeInvoke*/false);
 
                     return ret;
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c782b0b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
index 54bd543..9acab56 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
@@ -298,34 +298,16 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
             if (isDone())
                 return false;
 
-            if (err != null) {
-                tx.commitError(err);
-
-                boolean marked = tx.setRollbackOnly();
-
-                if (err instanceof IgniteTxRollbackCheckedException) {
-                    if (marked) {
-                        try {
-                            tx.rollback();
-                        }
-                        catch (IgniteCheckedException ex) {
-                            U.error(log, "Failed to automatically rollback transaction: " + tx, ex);
-                        }
-                    }
-                }
-                else if (tx.implicit() && tx.isSystemInvalidate()) { // Finish implicit transaction on heuristic error.
-                    try {
-                        tx.close();
-                    }
-                    catch (IgniteCheckedException ex) {
-                        U.error(log, "Failed to invalidate transaction: " + tx, ex);
-                    }
-                }
+            if (err != null)
+                tx.setRollbackOnly();
+
+            if (commit) {
+                if (tx.commitError() != null)
+                    err = tx.commitError();
+                else if (err != null)
+                    tx.commitError(err);
             }
 
-            if (commit && tx.commitError() != null)
-                err = tx.commitError();
-
             if (initialized() || err != null) {
                 if (tx.needCheckBackup()) {
                     assert tx.onePhaseCommit();
@@ -349,7 +331,15 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
 
                     finishOnePhase(commit);
 
-                    tx.tmFinish(commit);
+                    try {
+                        tx.tmFinish(commit);
+                    }
+                    catch (IgniteCheckedException e) {
+                        U.error(log, "Failed to finish tx: " + tx, e);
+
+                        if (err == null)
+                            err = e;
+                    }
                 }
 
                 if (super.onDone(tx0, err)) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c782b0b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index ed37059..0730300 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -231,7 +231,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override protected void sendFinishReply(boolean commit, @Nullable Throwable err) {
+    @Override protected void sendFinishReply(@Nullable Throwable err) {
         // We are in near transaction, do not send finish reply to local node.
     }
 
@@ -1062,50 +1062,48 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
             return new GridFinishedFuture<IgniteInternalTx>(this);
         }
 
-        final GridDhtTxFinishFuture fut = new GridDhtTxFinishFuture<>(cctx, this, /*commit*/true);
+        final GridDhtTxFinishFuture fut = new GridDhtTxFinishFuture<>(cctx, this, true);
 
         cctx.mvcc().addFuture(fut, fut.futureId());
 
         if (prep == null || prep.isDone()) {
             assert prep != null || optimistic();
 
+            IgniteCheckedException err = null;
+
             try {
                 if (prep != null)
                     prep.get(); // Check for errors of a parent future.
-
-                fut.finish();
-            }
-            catch (IgniteTxOptimisticCheckedException e) {
-                if (log.isDebugEnabled())
-                    log.debug("Failed optimistically to prepare transaction [tx=" + this + ", e=" + e + ']');
-
-                fut.onError(e);
             }
             catch (IgniteCheckedException e) {
-                U.error(log, "Failed to prepare transaction: " + this, e);
+                err = e;
 
-                fut.onError(e);
+                U.error(log, "Failed to prepare transaction: " + this, e);
             }
+
+            if (err != null)
+                fut.rollbackOnError(err);
+            else
+                fut.finish(true);
         }
         else
             prep.listen(new CI1<IgniteInternalFuture<?>>() {
                 @Override public void apply(IgniteInternalFuture<?> f) {
+                    IgniteCheckedException err = null;
+
                     try {
                         f.get(); // Check for errors of a parent future.
-
-                        fut.finish();
-                    }
-                    catch (IgniteTxOptimisticCheckedException e) {
-                        if (log.isDebugEnabled())
-                            log.debug("Failed optimistically to prepare transaction [tx=" + this + ", e=" + e + ']');
-
-                        fut.onError(e);
                     }
                     catch (IgniteCheckedException e) {
-                        U.error(log, "Failed to prepare transaction: " + this, e);
+                        err = e;
 
-                        fut.onError(e);
+                        U.error(log, "Failed to prepare transaction: " + this, e);
                     }
+
+                    if (err != null)
+                        fut.rollbackOnError(err);
+                    else
+                        fut.finish(true);
                 }
             });
 
@@ -1121,7 +1119,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
         if (log.isDebugEnabled())
             log.debug("Rolling back colocated tx locally: " + this);
 
-        final GridDhtTxFinishFuture fut = new GridDhtTxFinishFuture<>(cctx, this, /*commit*/false);
+        final GridDhtTxFinishFuture fut = new GridDhtTxFinishFuture<>(cctx, this, false);
 
         cctx.mvcc().addFuture(fut, fut.futureId());
 
@@ -1138,7 +1136,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
                         e.getMessage() + ']');
             }
 
-            fut.finish();
+            fut.finish(false);
         }
         else
             prep.listen(new CI1<IgniteInternalFuture<?>>() {
@@ -1151,7 +1149,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
                             e.getMessage() + ']');
                     }
 
-                    fut.finish();
+                    fut.finish(false);
                 }
             });
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c782b0b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index 2706d4d..eaf1c87 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -472,13 +472,8 @@ public class IgniteTxHandler {
                 req.last());
 
             if (tx.isRollbackOnly() && !tx.commitOnPrepare()) {
-                try {
-                    if (tx.state() != TransactionState.ROLLED_BACK && tx.state() != TransactionState.ROLLING_BACK)
-                        tx.rollback();
-                }
-                catch (IgniteCheckedException e) {
-                    U.error(log, "Failed to rollback transaction: " + tx, e);
-                }
+                if (tx.state() != TransactionState.ROLLED_BACK && tx.state() != TransactionState.ROLLING_BACK)
+                    tx.rollbackAsync();
             }
 
             final GridDhtTxLocal tx0 = tx;
@@ -872,7 +867,7 @@ public class IgniteTxHandler {
 
             U.error(log, "Failed completing transaction [commit=" + req.commit() + ", tx=" + tx + ']', e);
 
-            IgniteInternalFuture<IgniteInternalTx> res = null;
+            IgniteInternalFuture<IgniteInternalTx> res;
 
             IgniteInternalFuture<IgniteInternalTx> rollbackFut = tx.rollbackAsync();
 
@@ -884,7 +879,7 @@ public class IgniteTxHandler {
             if (e instanceof Error)
                 throw (Error)e;
 
-            return res == null ? new GridFinishedFuture<IgniteInternalTx>(e) : res;
+            return res;
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c782b0b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index ba44655..0327247 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -1000,8 +1000,9 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
      * Commits transaction to transaction manager. Used for one-phase commit transactions only.
      *
      * @param commit If {@code true} commits transaction, otherwise rollbacks.
+     * @throws IgniteCheckedException If failed.
      */
-    public void tmFinish(boolean commit) {
+    public void tmFinish(boolean commit) throws IgniteCheckedException {
         assert onePhaseCommit();
 
         if (DONE_FLAG_UPD.compareAndSet(this, 0, 1)) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c782b0b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
index bd8e18b..24f8ea9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
@@ -1192,8 +1192,9 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
      * Commits a transaction.
      *
      * @param tx Transaction to commit.
+     * @throws IgniteCheckedException If failed.
      */
-    public void commitTx(IgniteInternalTx tx) {
+    public void commitTx(IgniteInternalTx tx) throws IgniteCheckedException {
         assert tx != null;
         assert tx.state() == COMMITTING : "Invalid transaction state for commit from tm [state=" + tx.state() +
             ", expected=COMMITTING, tx=" + tx + ']';
@@ -1211,12 +1212,12 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
         Boolean committed = committed0 != null && !committed0.equals(Boolean.FALSE);
 
         // 1. Make sure that committed version has been recorded.
-        if (!((committed != null && committed) || tx.writeSet().isEmpty() || tx.isSystemInvalidate())) {
+        if (!(committed || tx.writeSet().isEmpty() || tx.isSystemInvalidate())) {
             uncommitTx(tx);
 
             tx.errorWhenCommitting();
 
-            throw new IgniteException("Missing commit version (consider increasing " +
+            throw new IgniteCheckedException("Missing commit version (consider increasing " +
                 IGNITE_MAX_COMPLETED_TX_COUNT + " system property) [ver=" + tx.xidVersion() +
                 ", tx=" + tx.getClass().getSimpleName() + ']');
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c782b0b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMissingCommitVersionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMissingCommitVersionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMissingCommitVersionSelfTest.java
index 19e49f3..ac56d18 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMissingCommitVersionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMissingCommitVersionSelfTest.java
@@ -43,7 +43,7 @@ public class GridCacheMissingCommitVersionSelfTest extends GridCommonAbstractTes
     private volatile boolean putFailed;
 
     /** */
-    private String maxCompletedTxCount;
+    private String maxCompletedTxCnt;
 
     /**
      */
@@ -53,7 +53,7 @@ public class GridCacheMissingCommitVersionSelfTest extends GridCommonAbstractTes
 
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration() throws Exception {
-        maxCompletedTxCount = System.getProperty(IGNITE_MAX_COMPLETED_TX_COUNT);
+        maxCompletedTxCnt = System.getProperty(IGNITE_MAX_COMPLETED_TX_COUNT);
 
         System.setProperty(IGNITE_MAX_COMPLETED_TX_COUNT, String.valueOf(5));
 
@@ -78,7 +78,7 @@ public class GridCacheMissingCommitVersionSelfTest extends GridCommonAbstractTes
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
-        System.setProperty(IGNITE_MAX_COMPLETED_TX_COUNT, maxCompletedTxCount != null ? maxCompletedTxCount : "");
+        System.setProperty(IGNITE_MAX_COMPLETED_TX_COUNT, maxCompletedTxCnt != null ? maxCompletedTxCnt : "");
 
         super.afterTest();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0c782b0b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxStoreExceptionAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxStoreExceptionAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxStoreExceptionAbstractSelfTest.java
index b65b441..795ab81 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxStoreExceptionAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxStoreExceptionAbstractSelfTest.java
@@ -357,6 +357,7 @@ public abstract class IgniteTxStoreExceptionAbstractSelfTest extends GridCacheAb
 
     /**
      * @param key Key.
+     * @param putBefore If {@code true} expects non-null values.
      * @throws Exception If failed.
      */
     private void checkValue(final Integer key, boolean putBefore) throws Exception {


[12/16] ignite git commit: IGNITE-1443: Implemented ContinuousQuery for C++

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/ignite_environment.cpp b/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
index 4947b94..b058f7c 100644
--- a/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
+++ b/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
@@ -18,6 +18,7 @@
 #include "ignite/impl/interop/interop_external_memory.h"
 #include "ignite/impl/binary/binary_reader_impl.h"
 #include "ignite/impl/ignite_environment.h"
+#include "ignite/cache/query/continuous/continuous_query.h"
 #include "ignite/binary/binary.h"
 #include "ignite/impl/binary/binary_type_updater_impl.h"
 
@@ -26,6 +27,7 @@ using namespace ignite::jni::java;
 using namespace ignite::impl::interop;
 using namespace ignite::impl::binary;
 using namespace ignite::binary;
+using namespace ignite::impl::cache::query::continuous;
 
 namespace ignite 
 {
@@ -36,6 +38,8 @@ namespace ignite
         */
         enum CallbackOp
         {
+            CONTINUOUS_QUERY_LISTENER_APPLY = 18,
+            CONTINUOUS_QUERY_FILTER_RELEASE = 21,
             REALLOC = 36,
             ON_START = 49,
             ON_STOP = 50 
@@ -50,11 +54,36 @@ namespace ignite
          */
         long long IGNITE_CALL InLongOutLong(void* target, int type, long long val)
         {
-            if (type == ON_STOP)
+            SharedPointer<IgniteEnvironment>* env = static_cast<SharedPointer<IgniteEnvironment>*>(target);
+
+            switch (type)
             {
-                SharedPointer<IgniteEnvironment>* ptr = static_cast<SharedPointer<IgniteEnvironment>*>(target);
+                case ON_STOP:
+                {
+                    delete env;
+
+                    break;
+                }
+
+                case CONTINUOUS_QUERY_LISTENER_APPLY:
+                {
+                    SharedPointer<InteropMemory> mem = env->Get()->GetMemory(val);
+
+                    env->Get()->OnContinuousQueryListenerApply(mem);
+
+                    break;
+                }
+
+                case CONTINUOUS_QUERY_FILTER_RELEASE:
+                {
+                    // No-op.
+                    break;
+                }
 
-                delete ptr;
+                default:
+                {
+                    break;
+                }
             }
 
             return 0;
@@ -73,26 +102,43 @@ namespace ignite
         long long IGNITE_CALL InLongLongLongObjectOutLong(void* target, int type, long long val1, long long val2, 
             long long val3, void* arg)
         {
-            if (type == ON_START)
-            {
-                SharedPointer<IgniteEnvironment>* ptr = static_cast<SharedPointer<IgniteEnvironment>*>(target);
+            SharedPointer<IgniteEnvironment>* env = static_cast<SharedPointer<IgniteEnvironment>*>(target);
 
-                ptr->Get()->OnStartCallback(val1, reinterpret_cast<jobject>(arg));
-            }
-            else if (type == REALLOC)
+            switch (type)
             {
-                SharedPointer<IgniteEnvironment>* env = static_cast<SharedPointer<IgniteEnvironment>*>(target);
+                case ON_START:
+                {
+                    env->Get()->OnStartCallback(val1, reinterpret_cast<jobject>(arg));
+
+                    break;
+                }
 
-                SharedPointer<InteropMemory> mem = env->Get()->GetMemory(val1);
+                case REALLOC:
+                {
+                    SharedPointer<InteropMemory> mem = env->Get()->GetMemory(val1);
 
-                mem.Get()->Reallocate(static_cast<int32_t>(val2));
+                    mem.Get()->Reallocate(static_cast<int32_t>(val2));
+
+                    break;
+                }
+
+                default:
+                {
+                    break;
+                }
             }
 
             return 0;
         }
 
-        IgniteEnvironment::IgniteEnvironment() : ctx(SharedPointer<JniContext>()), latch(new SingleLatch), name(0),
-            proc(), metaMgr(new BinaryTypeManager()), metaUpdater(0)
+        IgniteEnvironment::IgniteEnvironment() :
+            ctx(SharedPointer<JniContext>()),
+            latch(new SingleLatch),
+            name(0),
+            proc(),
+            metaMgr(new BinaryTypeManager()),
+            metaUpdater(0),
+            registry(DEFAULT_FAST_PATH_CONTAINERS_CAP, DEFAULT_SLOW_PATH_CONTAINERS_CAP)
         {
             // No-op.
         }
@@ -107,7 +153,7 @@ namespace ignite
 
         JniHandlers IgniteEnvironment::GetJniHandlers(SharedPointer<IgniteEnvironment>* target)
         {
-            JniHandlers hnds = JniHandlers();
+            JniHandlers hnds;
 
             hnds.target = target;
 
@@ -193,6 +239,11 @@ namespace ignite
                 ctx.Get()->ProcessorReleaseStart(proc.Get());
         }
 
+        HandleRegistry& IgniteEnvironment::GetHandleRegistry()
+        {
+            return registry;
+        }
+
         void IgniteEnvironment::OnStartCallback(long long memPtr, jobject proc)
         {
             this->proc = jni::JavaGlobalRef(*ctx.Get(), proc);
@@ -212,6 +263,23 @@ namespace ignite
             else
                 name = 0;
         }
+
+        void IgniteEnvironment::OnContinuousQueryListenerApply(SharedPointer<InteropMemory>& mem)
+        {
+            InteropInputStream stream(mem.Get());
+            BinaryReaderImpl reader(&stream);
+
+            int64_t qryHandle = reader.ReadInt64();
+
+            ContinuousQueryImplBase* contQry = reinterpret_cast<ContinuousQueryImplBase*>(registry.Get(qryHandle).Get());
+
+            if (contQry)
+            {
+                BinaryRawReader rawReader(&reader);
+
+                contQry->ReadAndProcessEvents(rawReader);
+            }
+        }
     }
 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/examples/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/Makefile.am b/modules/platforms/cpp/examples/Makefile.am
index cda5132..e1c7905 100644
--- a/modules/platforms/cpp/examples/Makefile.am
+++ b/modules/platforms/cpp/examples/Makefile.am
@@ -21,5 +21,6 @@ SUBDIRS = \
     putget-example \
     odbc-example \
     query-example \
+	continuous-query-example \
     include
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/examples/configure.ac
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/configure.ac b/modules/platforms/cpp/examples/configure.ac
index 6ddd7c8..d167cf0 100644
--- a/modules/platforms/cpp/examples/configure.ac
+++ b/modules/platforms/cpp/examples/configure.ac
@@ -56,6 +56,7 @@ AC_CONFIG_FILES([ \
     putget-example/Makefile \
     odbc-example/Makefile \
     query-example/Makefile \
+    continuous-query-example/Makefile \
 ])
 
 AC_OUTPUT

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/examples/continuous-query-example/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/continuous-query-example/Makefile.am b/modules/platforms/cpp/examples/continuous-query-example/Makefile.am
new file mode 100644
index 0000000..2566689
--- /dev/null
+++ b/modules/platforms/cpp/examples/continuous-query-example/Makefile.am
@@ -0,0 +1,58 @@
+##
+## Licensed to the Apache Software Foundation (ASF) under one or more
+## contributor license agreements.  See the NOTICE file distributed with
+## this work for additional information regarding copyright ownership.
+## The ASF licenses this file to You under the Apache License, Version 2.0
+## (the "License"); you may not use this file except in compliance with
+## the License.  You may obtain a copy of the License at
+##
+##      http://www.apache.org/licenses/LICENSE-2.0
+##
+## Unless required by applicable law or agreed to in writing, software
+## distributed under the License is distributed on an "AS IS" BASIS,
+## WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+## See the License for the specific language governing permissions and
+## limitations under the License.
+##
+
+ACLOCAL_AMFLAGS =-I m4
+
+noinst_PROGRAMS = ignite-continuous-query-example
+
+AM_CPPFLAGS = \
+    -I@top_srcdir@/include \
+    -I@top_srcdir@/../core/include \
+    -I@top_srcdir@/../core/os/linux/include \
+    -I@top_srcdir@/../common/include \
+    -I@top_srcdir@/../common/os/linux/include \
+    -I@top_srcdir@/../binary/include \
+    -I@top_srcdir@/../jni/include \
+    -I@top_srcdir@/../jni/os/linux/include \
+    -I$(JAVA_HOME)/include \
+    -I$(JAVA_HOME)/include/linux \
+    -DIGNITE_IMPL \
+    -D__STDC_LIMIT_MACROS \
+    -D__STDC_CONSTANT_MACROS
+
+AM_CXXFLAGS = \
+    -Wall \
+    -std=c++0x
+
+ignite_continuous_query_example_LDADD = \
+    @top_srcdir@/../core/libignite.la \
+    -lpthread
+
+ignite_continuous_query_example_LDFLAGS = \
+    -static-libtool-libs
+
+ignite_continuous_query_example_SOURCES = \
+    src/continuous_query_example.cpp
+
+run-check: check
+	./ignite-continuous-query-example -p
+
+clean-local: clean-check
+	$(RM) *.gcno *.gcda
+
+clean-check:
+	$(RM) $(ignite_continuous_query_example_OBJECTS)

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/examples/continuous-query-example/config/continuous-query-example.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/continuous-query-example/config/continuous-query-example.xml b/modules/platforms/cpp/examples/continuous-query-example/config/continuous-query-example.xml
new file mode 100644
index 0000000..bdc1e92
--- /dev/null
+++ b/modules/platforms/cpp/examples/continuous-query-example/config/continuous-query-example.xml
@@ -0,0 +1,52 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+
+        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <!--
+                        Ignite provides several options for automatic discovery that can be used
+                        instead os static IP based discovery.
+                    -->
+                    <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
+                    <!--<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">-->
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47550..47551</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/examples/continuous-query-example/project/vs/continuous-query-example.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/continuous-query-example/project/vs/continuous-query-example.vcxproj b/modules/platforms/cpp/examples/continuous-query-example/project/vs/continuous-query-example.vcxproj
new file mode 100644
index 0000000..dfc5f9d
--- /dev/null
+++ b/modules/platforms/cpp/examples/continuous-query-example/project/vs/continuous-query-example.vcxproj
@@ -0,0 +1,110 @@
+\ufeff<?xml version="1.0" encoding="utf-8"?>
+<Project DefaultTargets="Build" ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
+  <ItemGroup Label="ProjectConfigurations">
+    <ProjectConfiguration Include="Release|Win32">
+      <Configuration>Release</Configuration>
+      <Platform>Win32</Platform>
+    </ProjectConfiguration>
+    <ProjectConfiguration Include="Release|x64">
+      <Configuration>Release</Configuration>
+      <Platform>x64</Platform>
+    </ProjectConfiguration>
+  </ItemGroup>
+  <PropertyGroup Label="Globals">
+    <RootNamespace>igniteexamples</RootNamespace>
+    <ProjectGuid>{73BB124A-0CD4-4961-A6CD-61F9C71028A6}</ProjectGuid>
+  </PropertyGroup>
+  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.Default.props" />
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="Configuration">
+    <ConfigurationType>Application</ConfigurationType>
+    <UseDebugLibraries>false</UseDebugLibraries>
+    <PlatformToolset>v100</PlatformToolset>
+    <WholeProgramOptimization>true</WholeProgramOptimization>
+    <CharacterSet>Unicode</CharacterSet>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="Configuration">
+    <ConfigurationType>Application</ConfigurationType>
+    <UseDebugLibraries>false</UseDebugLibraries>
+    <PlatformToolset>v100</PlatformToolset>
+    <WholeProgramOptimization>true</WholeProgramOptimization>
+    <CharacterSet>Unicode</CharacterSet>
+  </PropertyGroup>
+  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.props" />
+  <ImportGroup Label="ExtensionSettings">
+  </ImportGroup>
+  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="PropertySheets">
+    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
+  </ImportGroup>
+  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="PropertySheets">
+    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
+  </ImportGroup>
+  <PropertyGroup Label="UserMacros" />
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
+    <LinkIncremental>false</LinkIncremental>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
+    <LinkIncremental>false</LinkIncremental>
+  </PropertyGroup>
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
+    <ClCompile>
+      <WarningLevel>Level3</WarningLevel>
+      <PrecompiledHeader>
+      </PrecompiledHeader>
+      <Optimization>MaxSpeed</Optimization>
+      <FunctionLevelLinking>true</FunctionLevelLinking>
+      <IntrinsicFunctions>true</IntrinsicFunctions>
+      <PreprocessorDefinitions>WIN32;NDEBUG;_CONSOLE;_LIB;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <AdditionalIncludeDirectories>$(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;..\..\..\include;..\..\..\..\jni\os\win\include;..\..\..\..\jni\include;..\..\..\..\common\os\win\include;..\..\..\..\common\include;..\..\..\..\binary\include;..\..\..\..\core\os\win\include;..\..\..\..\core\include;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
+    </ClCompile>
+    <Link>
+      <SubSystem>Console</SubSystem>
+      <GenerateDebugInformation>true</GenerateDebugInformation>
+      <EnableCOMDATFolding>true</EnableCOMDATFolding>
+      <OptimizeReferences>true</OptimizeReferences>
+      <AdditionalDependencies>jvm.lib;ignite.jni.lib;ignite.binary.lib;ignite.core.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalLibraryDirectories>..\..\..\..\project\vs\$(Platform)\$(Configuration)\;$(JAVA_HOME)\lib;%(AdditionalLibraryDirectories)</AdditionalLibraryDirectories>
+    </Link>
+    <PostBuildEvent>
+      <Command>copy "$(ProjectDir)..\..\..\..\project\vs\$(Platform)\$(Configuration)\ignite.common.dll" "$(OutDir)"
+copy "$(ProjectDir)..\..\..\..\project\vs\$(Platform)\$(Configuration)\ignite.core.dll" "$(OutDir)"</Command>
+    </PostBuildEvent>
+  </ItemDefinitionGroup>
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
+    <ClCompile>
+      <WarningLevel>Level3</WarningLevel>
+      <PrecompiledHeader>
+      </PrecompiledHeader>
+      <Optimization>MaxSpeed</Optimization>
+      <FunctionLevelLinking>true</FunctionLevelLinking>
+      <IntrinsicFunctions>true</IntrinsicFunctions>
+      <PreprocessorDefinitions>WIN32;NDEBUG;_CONSOLE;_LIB;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <AdditionalIncludeDirectories>$(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;..\..\..\include;..\..\..\..\jni\os\win\include;..\..\..\..\jni\include;..\..\..\..\common\os\win\include;..\..\..\..\common\include;..\..\..\..\binary\include;..\..\..\..\core\os\win\include;..\..\..\..\core\include;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
+    </ClCompile>
+    <Link>
+      <SubSystem>Console</SubSystem>
+      <GenerateDebugInformation>true</GenerateDebugInformation>
+      <EnableCOMDATFolding>true</EnableCOMDATFolding>
+      <OptimizeReferences>true</OptimizeReferences>
+      <AdditionalDependencies>jvm.lib;ignite.jni.lib;ignite.binary.lib;ignite.core.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalLibraryDirectories>..\..\..\..\project\vs\$(Platform)\$(Configuration)\;$(JAVA_HOME)\lib;%(AdditionalLibraryDirectories)</AdditionalLibraryDirectories>
+    </Link>
+    <PostBuildEvent>
+      <Command>copy "$(ProjectDir)..\..\..\..\project\vs\$(Platform)\$(Configuration)\ignite.common.dll" "$(OutDir)"
+copy "$(ProjectDir)..\..\..\..\project\vs\$(Platform)\$(Configuration)\ignite.core.dll" "$(OutDir)"</Command>
+    </PostBuildEvent>
+  </ItemDefinitionGroup>
+  <ItemGroup>
+    <ClCompile Include="..\..\src\continuous_query_example.cpp" />
+  </ItemGroup>
+  <ItemGroup>
+    <ClInclude Include="..\..\..\include\ignite\examples\address.h" />
+    <ClInclude Include="..\..\..\include\ignite\examples\organization.h" />
+    <ClInclude Include="..\..\..\include\ignite\examples\person.h" />
+  </ItemGroup>
+  <ItemGroup>
+    <None Include="..\..\config\continuous-query-example.xml" />
+  </ItemGroup>
+  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.targets" />
+  <ImportGroup Label="ExtensionTargets">
+  </ImportGroup>
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/examples/continuous-query-example/project/vs/continuous-query-example.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/continuous-query-example/project/vs/continuous-query-example.vcxproj.filters b/modules/platforms/cpp/examples/continuous-query-example/project/vs/continuous-query-example.vcxproj.filters
new file mode 100644
index 0000000..cf3bca9
--- /dev/null
+++ b/modules/platforms/cpp/examples/continuous-query-example/project/vs/continuous-query-example.vcxproj.filters
@@ -0,0 +1,35 @@
+\ufeff<?xml version="1.0" encoding="utf-8"?>
+<Project ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
+  <ItemGroup>
+    <ClInclude Include="..\..\..\include\ignite\examples\address.h">
+      <Filter>Header Files</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\..\include\ignite\examples\organization.h">
+      <Filter>Header Files</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\..\include\ignite\examples\person.h">
+      <Filter>Header Files</Filter>
+    </ClInclude>
+  </ItemGroup>
+  <ItemGroup>
+    <Filter Include="Source Files">
+      <UniqueIdentifier>{35cb32b7-bf2e-440f-9b32-80d392d81847}</UniqueIdentifier>
+    </Filter>
+    <Filter Include="Header Files">
+      <UniqueIdentifier>{b355095f-b4e2-4324-9516-854828c876ff}</UniqueIdentifier>
+    </Filter>
+    <Filter Include="Config">
+      <UniqueIdentifier>{3799efd0-3cfe-47e2-9e9e-a51b25bf40ef}</UniqueIdentifier>
+    </Filter>
+  </ItemGroup>
+  <ItemGroup>
+    <ClCompile Include="..\..\src\continuous_query_example.cpp">
+      <Filter>Source Files</Filter>
+    </ClCompile>
+  </ItemGroup>
+  <ItemGroup>
+    <None Include="..\..\config\continuous-query-example.xml">
+      <Filter>Config</Filter>
+    </None>
+  </ItemGroup>
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/examples/continuous-query-example/src/continuous_query_example.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/continuous-query-example/src/continuous_query_example.cpp b/modules/platforms/cpp/examples/continuous-query-example/src/continuous_query_example.cpp
new file mode 100644
index 0000000..b08d4b8
--- /dev/null
+++ b/modules/platforms/cpp/examples/continuous-query-example/src/continuous_query_example.cpp
@@ -0,0 +1,142 @@
+/*
+ * 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.
+ */
+
+#include <stdint.h>
+#include <iostream>
+
+#include "ignite/ignition.h"
+#include "ignite/cache/query/continuous/continuous_query.h"
+
+#include "ignite/examples/organization.h"
+#include "ignite/examples/person.h"
+
+using namespace ignite;
+using namespace cache;
+using namespace query;
+
+using namespace examples;
+
+/** Cache name. */
+const char* CACHE_NAME = "cpp_cache_continuous_query";
+
+/*
+ * Listener class.
+ */
+template<typename K, typename V>
+class Listener : public event::CacheEntryEventListener<K, V>
+{
+public:
+    /*
+     * Default constructor.
+     */
+    Listener()
+    {
+        // No-op.
+    }
+
+    /**
+     * Event callback.
+     *
+     * @param evts Events.
+     * @param num Events number.
+     */
+    virtual void OnEvent(const CacheEntryEvent<K, V>* evts, uint32_t num)
+    {
+        for (uint32_t i = 0; i < num; ++i)
+        {
+            std::cout << "Queried entry [key=" << (evts[i].HasValue() ? evts[i].GetKey() : K())
+                      << ", val=" << (evts[i].HasValue() ? evts[i].GetValue() : V()) << ']'
+                      << std::endl;
+        }
+    }
+};
+
+int main()
+{
+    IgniteConfiguration cfg;
+
+    cfg.springCfgPath = "platforms/cpp/examples/continuous-query-example/config/continuous-query-example.xml";
+
+    try
+    {
+        // Start a node.
+        Ignite ignite = Ignition::Start(cfg);
+
+        std::cout << std::endl;
+        std::cout << ">>> Cache continuous query example started." << std::endl;
+        std::cout << std::endl;
+
+        // Get cache instance.
+        Cache<int32_t, std::string> cache = ignite.GetOrCreateCache<int32_t, std::string>(CACHE_NAME);
+
+        cache.Clear();
+
+        const int32_t keyCnt = 20;
+
+        for (int32_t i = 0; i < keyCnt; ++i)
+        {
+            std::stringstream builder;
+
+            builder << i;
+
+            cache.Put(i, builder.str());
+        }
+
+        // Declaring listener.
+        Listener<int, std::string> listener;
+
+        // Declaring continuous query.
+        continuous::ContinuousQuery<int, std::string> qry(MakeReference(listener));
+
+        {
+            // Continous query scope. Query is closed when scope is left.
+            continuous::ContinuousQueryHandle<int, std::string> handle = cache.QueryContinuous(qry);
+
+            // Add a few more keys and watch more query notifications.
+            for (int32_t i = keyCnt; i < keyCnt + 5; ++i)
+            {
+                std::stringstream builder;
+
+                builder << i;
+
+                cache.Put(i, builder.str());
+            }
+
+            // Let user wait while callback is notified about remaining puts.
+            std::cout << std::endl;
+            std::cout << ">>> Press 'Enter' to continue..." << std::endl;
+            std::cout << std::endl;
+
+            std::cin.get();
+        }
+
+        // Stop node.
+        Ignition::StopAll(false);
+    }
+    catch (IgniteError& err)
+    {
+        std::cout << "An error occurred: " << err.GetText() << std::endl;
+    }
+
+    std::cout << std::endl;
+    std::cout << ">>> Example finished, press 'Enter' to exit ..." << std::endl;
+    std::cout << std::endl;
+
+    std::cin.get();
+
+    return 0;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/examples/include/ignite/examples/person.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/include/ignite/examples/person.h b/modules/platforms/cpp/examples/include/ignite/examples/person.h
index 2c92660..86c51c8 100644
--- a/modules/platforms/cpp/examples/include/ignite/examples/person.h
+++ b/modules/platforms/cpp/examples/include/ignite/examples/person.h
@@ -46,7 +46,7 @@ namespace ignite
                 // No-op.
             }
 
-            std::string ToString()
+            std::string ToString() const
             {
                 std::ostringstream oss;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/examples/project/vs/ignite-examples.sln
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/project/vs/ignite-examples.sln b/modules/platforms/cpp/examples/project/vs/ignite-examples.sln
index 89f609f..d839f09 100644
--- a/modules/platforms/cpp/examples/project/vs/ignite-examples.sln
+++ b/modules/platforms/cpp/examples/project/vs/ignite-examples.sln
@@ -7,6 +7,8 @@ Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "odbc-example", "..\..\odbc-
 EndProject
 Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "query-example", "..\..\query-example\project\vs\query-example.vcxproj", "{9FB34AB4-01DD-4C6F-99BF-681019D0E4DD}"
 EndProject
+Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "continuous-query-example", "..\..\continuous-query-example\project\vs\continuous-query-example.vcxproj", "{73BB124A-0CD4-4961-A6CD-61F9C71028A6}"
+EndProject
 Global
 	GlobalSection(SolutionConfigurationPlatforms) = preSolution
 		Release|x64 = Release|x64
@@ -25,6 +27,10 @@ Global
 		{9FB34AB4-01DD-4C6F-99BF-681019D0E4DD}.Release|x64.Build.0 = Release|x64
 		{9FB34AB4-01DD-4C6F-99BF-681019D0E4DD}.Release|x86.ActiveCfg = Release|Win32
 		{9FB34AB4-01DD-4C6F-99BF-681019D0E4DD}.Release|x86.Build.0 = Release|Win32
+		{73BB124A-0CD4-4961-A6CD-61F9C71028A6}.Release|x64.ActiveCfg = Release|x64
+		{73BB124A-0CD4-4961-A6CD-61F9C71028A6}.Release|x64.Build.0 = Release|x64
+		{73BB124A-0CD4-4961-A6CD-61F9C71028A6}.Release|x86.ActiveCfg = Release|Win32
+		{73BB124A-0CD4-4961-A6CD-61F9C71028A6}.Release|x86.Build.0 = Release|Win32
 	EndGlobalSection
 	GlobalSection(SolutionProperties) = preSolution
 		HideSolutionNode = FALSE

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/jni/include/ignite/jni/java.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/include/ignite/jni/java.h b/modules/platforms/cpp/jni/include/ignite/jni/java.h
index 442cc10..6289d73 100644
--- a/modules/platforms/cpp/jni/include/ignite/jni/java.h
+++ b/modules/platforms/cpp/jni/include/ignite/jni/java.h
@@ -389,7 +389,7 @@ namespace ignite
                 void TargetListenFutureForOperation(jobject obj, long long futId, int typ, int opId);
 
                 jobject CacheOutOpQueryCursor(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL);
-                jobject CacheOutOpContinuousQuery(jobject obj, int type, long long memPtr);
+                jobject CacheOutOpContinuousQuery(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL);
 
                 jobject Acquire(jobject obj);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/jni/src/java.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/src/java.cpp b/modules/platforms/cpp/jni/src/java.cpp
index 698263d..e6f2f88 100644
--- a/modules/platforms/cpp/jni/src/java.cpp
+++ b/modules/platforms/cpp/jni/src/java.cpp
@@ -1357,13 +1357,13 @@ namespace ignite
                 return LocalToGlobal(env, res);
             }
 
-            jobject JniContext::CacheOutOpContinuousQuery(jobject obj, int type, long long memPtr) {
+            jobject JniContext::CacheOutOpContinuousQuery(jobject obj, int type, long long memPtr, JniErrorInfo* err) {
                 JNIEnv* env = Attach();
 
                 jobject res = env->CallObjectMethod(
                     obj, jvm->GetMembers().m_PlatformTarget_inStreamOutObject, type, memPtr);
 
-                ExceptionCheck(env);
+                ExceptionCheck(env, err);
 
                 return LocalToGlobal(env, res);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
index 753ae4c..6f85896 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
@@ -97,7 +97,7 @@
     </ClCompile>
     <Link>
       <GenerateDebugInformation>true</GenerateDebugInformation>
-      <AdditionalDependencies>$(JAVA_HOME)\lib\jvm.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_unit_test_framework-vc100-mt-gd-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalDependencies>$(JAVA_HOME)\lib\jvm.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_unit_test_framework-vc100-mt-gd-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
     </Link>
   </ItemDefinitionGroup>
   <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'">
@@ -130,7 +130,7 @@
       <GenerateDebugInformation>true</GenerateDebugInformation>
       <EnableCOMDATFolding>true</EnableCOMDATFolding>
       <OptimizeReferences>true</OptimizeReferences>
-      <AdditionalDependencies>$(JAVA_HOME)\lib\jvm.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_unit_test_framework-vc100-mt-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalDependencies>$(JAVA_HOME)\lib\jvm.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_unit_test_framework-vc100-mt-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
     </Link>
   </ItemDefinitionGroup>
   <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/project/vs/ignite.slnrel
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/project/vs/ignite.slnrel b/modules/platforms/cpp/project/vs/ignite.slnrel
index 1d874a8..6a37bf4 100644
--- a/modules/platforms/cpp/project/vs/ignite.slnrel
+++ b/modules/platforms/cpp/project/vs/ignite.slnrel
@@ -2,6 +2,9 @@
 Microsoft Visual Studio Solution File, Format Version 11.00
 # Visual Studio 2010
 Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "core", "..\..\core\project\vs\core.vcxproj", "{E2DEA693-F2EA-43C2-A813-053378F6E4DB}"
+	ProjectSection(ProjectDependencies) = postProject
+		{4F15669B-92EB-49F0-B774-8F19BAE0B960} = {4F15669B-92EB-49F0-B774-8F19BAE0B960}
+	EndProjectSection
 EndProject
 Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "ignite", "..\..\ignite\project\vs\ignite.vcxproj", "{69688B4D-3EE0-43F5-A1C6-29B5D2DDE949}"
 	ProjectSection(ProjectDependencies) = postProject

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/project/vs/ignite_x86.slnrel
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/project/vs/ignite_x86.slnrel b/modules/platforms/cpp/project/vs/ignite_x86.slnrel
index 4785579..abd0ad1 100644
--- a/modules/platforms/cpp/project/vs/ignite_x86.slnrel
+++ b/modules/platforms/cpp/project/vs/ignite_x86.slnrel
@@ -2,6 +2,9 @@
 Microsoft Visual Studio Solution File, Format Version 11.00
 # Visual Studio 2010
 Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "core", "..\..\core\project\vs\core.vcxproj", "{E2DEA693-F2EA-43C2-A813-053378F6E4DB}"
+	ProjectSection(ProjectDependencies) = postProject
+		{4F15669B-92EB-49F0-B774-8F19BAE0B960} = {4F15669B-92EB-49F0-B774-8F19BAE0B960}
+	EndProjectSection
 EndProject
 Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "ignite", "..\..\ignite\project\vs\ignite.vcxproj", "{69688B4D-3EE0-43F5-A1C6-29B5D2DDE949}"
 	ProjectSection(ProjectDependencies) = postProject


[07/16] ignite git commit: ignite-4371 Neat TX finish request processing may fall into sync wait of dht finish response - enabled striped pool usage for GridDistributedTxFinishResponse - convert exceptions from CacheStoreSessionListener.onSessionStart in

Posted by vo...@apache.org.
ignite-4371 Neat TX finish request processing may fall into sync wait of dht finish response
- enabled striped pool usage for GridDistributedTxFinishResponse
- convert exceptions from CacheStoreSessionListener.onSessionStart into IgniteCheckedException
- should not use IgniteTxLocalAdapter.commitErr for rollback exceptions (otherwise cyclic reference in exception->cause is possible, this leads to infinite loop in 'X.hasCause')


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

Branch: refs/heads/ignite-2.0
Commit: cf7b6a0e9bc6fb48d6268bfa40e1aedc9f8c9252
Parents: fcf5d9a
Author: sboikov <sb...@gridgain.com>
Authored: Mon Dec 19 14:49:25 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Dec 19 14:49:25 2016 +0300

----------------------------------------------------------------------
 .../GridDistributedTxFinishResponse.java        |  6 ----
 .../cache/store/CacheStoreManager.java          |  4 ++-
 .../store/GridCacheStoreManagerAdapter.java     | 34 ++++++++++++--------
 .../cache/store/GridCacheWriteBehindStore.java  |  6 ++--
 .../transactions/IgniteTxLocalAdapter.java      | 27 ++++++----------
 5 files changed, 36 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cf7b6a0e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
index c5cf332..109d665 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
@@ -86,12 +86,6 @@ public class GridDistributedTxFinishResponse extends GridCacheMessage {
     }
 
     /** {@inheritDoc} */
-    @Override public int partition() {
-        // TODO https://issues.apache.org/jira/browse/IGNITE-4371
-        return Integer.MIN_VALUE;
-    }
-
-    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cf7b6a0e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
index 67c9334..8d6b63d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
@@ -171,8 +171,10 @@ public interface CacheStoreManager<K, V> extends GridCacheManager<K, V> {
 
     /**
      * End session initiated by write-behind store.
+     *
+     * @throws IgniteCheckedException If failed.
      */
-    public void writeBehindSessionInit();
+    public void writeBehindSessionInit() throws IgniteCheckedException;
 
     /**
      * End session initiated by write-behind store.

http://git-wip-us.apache.org/repos/asf/ignite/blob/cf7b6a0e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index 024375e..8b93afa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -288,7 +288,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override @Nullable public Object load(@Nullable IgniteInternalTx tx, KeyCacheObject key)
+    @Override @Nullable public final Object load(@Nullable IgniteInternalTx tx, KeyCacheObject key)
         throws IgniteCheckedException {
         return loadFromStore(tx, key, true);
     }
@@ -377,7 +377,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public void localStoreLoadAll(@Nullable IgniteInternalTx tx, Collection keys, GridInClosure3 vis)
+    @Override public final void localStoreLoadAll(@Nullable IgniteInternalTx tx, Collection keys, GridInClosure3 vis)
         throws IgniteCheckedException {
         assert store != null;
         assert locStore;
@@ -386,7 +386,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean loadAll(@Nullable IgniteInternalTx tx, Collection keys, IgniteBiInClosure vis)
+    @Override public final boolean loadAll(@Nullable IgniteInternalTx tx, Collection keys, IgniteBiInClosure vis)
         throws IgniteCheckedException {
         if (store != null) {
             loadAllFromStore(tx, keys, vis, null);
@@ -499,7 +499,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean loadCache(final GridInClosure3 vis, Object[] args) throws IgniteCheckedException {
+    @Override public final boolean loadCache(final GridInClosure3 vis, Object[] args) throws IgniteCheckedException {
         if (store != null) {
             if (log.isDebugEnabled())
                 log.debug("Loading all values from store.");
@@ -554,7 +554,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean put(@Nullable IgniteInternalTx tx, Object key, Object val, GridCacheVersion ver)
+    @Override public final boolean put(@Nullable IgniteInternalTx tx, Object key, Object val, GridCacheVersion ver)
         throws IgniteCheckedException {
         if (store != null) {
             // Never persist internal keys.
@@ -599,7 +599,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean putAll(@Nullable IgniteInternalTx tx, Map map) throws IgniteCheckedException {
+    @Override public final boolean putAll(@Nullable IgniteInternalTx tx, Map map) throws IgniteCheckedException {
         if (F.isEmpty(map))
             return true;
 
@@ -658,7 +658,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean remove(@Nullable IgniteInternalTx tx, Object key) throws IgniteCheckedException {
+    @Override public final boolean remove(@Nullable IgniteInternalTx tx, Object key) throws IgniteCheckedException {
         if (store != null) {
             // Never remove internal key from store as it is never persisted.
             if (key instanceof GridCacheInternal)
@@ -701,7 +701,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean removeAll(@Nullable IgniteInternalTx tx, Collection keys) throws IgniteCheckedException {
+    @Override public final boolean removeAll(@Nullable IgniteInternalTx tx, Collection keys) throws IgniteCheckedException {
         if (F.isEmpty(keys))
             return true;
 
@@ -763,7 +763,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last) throws IgniteCheckedException {
+    @Override public final void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last) throws IgniteCheckedException {
         assert store != null;
 
         sessionInit0(tx);
@@ -807,7 +807,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public void writeBehindSessionInit() {
+    @Override public void writeBehindSessionInit() throws IgniteCheckedException {
         sessionInit0(null);
     }
 
@@ -818,8 +818,9 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
     /**
      * @param tx Current transaction.
+     * @throws IgniteCheckedException If failed.
      */
-    private void sessionInit0(@Nullable IgniteInternalTx tx) {
+    private void sessionInit0(@Nullable IgniteInternalTx tx) throws IgniteCheckedException {
         assert sesHolder != null;
 
         SessionData ses;
@@ -841,9 +842,14 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
         sesHolder.set(ses);
 
-        if (sesLsnrs != null && !ses.started(this)) {
-            for (CacheStoreSessionListener lsnr : sesLsnrs)
-                lsnr.onSessionStart(locSes);
+        try {
+            if (sesLsnrs != null && !ses.started(this)) {
+                for (CacheStoreSessionListener lsnr : sesLsnrs)
+                    lsnr.onSessionStart(locSes);
+            }
+        }
+        catch (Exception e) {
+            throw new IgniteCheckedException("Failed to start store session: " + e, e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cf7b6a0e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
index 858d9a7..f5c0c12 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
@@ -673,10 +673,10 @@ public class GridCacheWriteBehindStore<K, V> implements CacheStore<K, V>, Lifecy
         Map<K, Entry<? extends K, ? extends  V>> vals,
         boolean initSes) {
 
-        if (initSes && storeMgr != null)
-            storeMgr.writeBehindSessionInit();
-
         try {
+            if (initSes && storeMgr != null)
+                storeMgr.writeBehindSessionInit();
+
             boolean threwEx = true;
 
             try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/cf7b6a0e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 0327247..e1479f9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -1063,7 +1063,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
             setRollbackOnly();
 
             throw new IgniteCheckedException("Invalid transaction state for rollback [state=" + state +
-                ", tx=" + this + ']', commitErr);
+                ", tx=" + this + ']');
         }
 
         if (near()) {
@@ -1074,28 +1074,21 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
         }
 
         if (DONE_FLAG_UPD.compareAndSet(this, 0, 1)) {
-            try {
-                cctx.tm().rollbackTx(this);
+            cctx.tm().rollbackTx(this);
 
-                if (!internal()) {
-                    Collection<CacheStoreManager> stores = txState.stores(cctx);
+            if (!internal()) {
+                Collection<CacheStoreManager> stores = txState.stores(cctx);
 
-                    if (stores != null && !stores.isEmpty()) {
-                        assert isWriteToStoreFromDhtValid(stores) :
-                            "isWriteToStoreFromDht can't be different within one transaction";
+                if (stores != null && !stores.isEmpty()) {
+                    assert isWriteToStoreFromDhtValid(stores) :
+                        "isWriteToStoreFromDht can't be different within one transaction";
 
-                        boolean isWriteToStoreFromDht = F.first(stores).isWriteToStoreFromDht();
+                    boolean isWriteToStoreFromDht = F.first(stores).isWriteToStoreFromDht();
 
-                        if (stores != null && !stores.isEmpty() && (near() || isWriteToStoreFromDht))
-                            sessionEnd(stores, false);
-                    }
+                    if (!stores.isEmpty() && (near() || isWriteToStoreFromDht))
+                        sessionEnd(stores, false);
                 }
             }
-            catch (Error | IgniteCheckedException | RuntimeException e) {
-                U.addLastCause(e, commitErr, log);
-
-                throw e;
-            }
         }
     }
 


[16/16] ignite git commit: TcpDiscoverySpi: need check result of sock.getSendBufferSize().

Posted by vo...@apache.org.
TcpDiscoverySpi: need check result of sock.getSendBufferSize().


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

Branch: refs/heads/ignite-2.0
Commit: 0a336445367fb2cec202c431a8614c185da9a3ca
Parents: 854d1a5
Author: sboikov <sb...@gridgain.com>
Authored: Wed Dec 21 13:28:25 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Dec 21 13:28:25 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/spi/discovery/tcp/ServerImpl.java   |  2 +-
 .../ignite/spi/discovery/tcp/TcpDiscoverySpi.java     | 14 +++++++++++++-
 2 files changed, 14 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0a336445/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 50fa3bd..7cc35e4 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -2771,7 +2771,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                                 sock = spi.openSocket(addr, timeoutHelper);
 
-                                out = new BufferedOutputStream(sock.getOutputStream(), sock.getSendBufferSize());
+                                out = spi.socketStream(sock);
 
                                 openSock = true;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0a336445/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index 45933e1..db85cc4 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -1246,6 +1246,18 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     }
 
     /**
+     * @param sock Socket.
+     * @return Buffered stream wrapping socket stream.
+     * @throws IOException If failed.
+     */
+    final BufferedOutputStream socketStream(Socket sock) throws IOException {
+        int bufSize = sock.getSendBufferSize();
+
+        return bufSize > 0 ? new BufferedOutputStream(sock.getOutputStream(), bufSize) :
+            new BufferedOutputStream(sock.getOutputStream());
+    }
+
+    /**
      * Connects to remote address sending {@code U.IGNITE_HEADER} when connection is established.
      *
      * @param sock Socket bound to a local host address.
@@ -1351,7 +1363,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
      */
     protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg, long timeout) throws IOException,
         IgniteCheckedException {
-        writeToSocket(sock, new BufferedOutputStream(sock.getOutputStream(), sock.getSendBufferSize()), msg, timeout);
+        writeToSocket(sock, socketStream(sock), msg, timeout);
     }
 
     /**


[06/16] ignite git commit: IGNITE-3966: Hadoop: add Hadoop native library to Java arguments automatically. This closes #1320.

Posted by vo...@apache.org.
IGNITE-3966: Hadoop: add Hadoop native library to Java arguments automatically. This closes #1320.


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

Branch: refs/heads/ignite-2.0
Commit: fcf5d9ac714df93c615f97fdc306913bf7ed7e50
Parents: 1e97d78
Author: Ivan Veselovskiy <iv...@gridgain.com>
Authored: Mon Dec 19 11:57:00 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Dec 19 11:57:00 2016 +0300

----------------------------------------------------------------------
 bin/ignite.sh | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fcf5d9ac/bin/ignite.sh
----------------------------------------------------------------------
diff --git a/bin/ignite.sh b/bin/ignite.sh
index 05d2d5f..efa3715 100755
--- a/bin/ignite.sh
+++ b/bin/ignite.sh
@@ -128,6 +128,15 @@ if [ "${ENABLE_ASSERTIONS}" = "1" ]; then
 fi
 
 #
+# If this is a Hadoop edition, and HADOOP_HOME set, add the native library location:
+#
+if [ -d "${IGNITE_HOME}/libs/ignite-hadoop/" ] && [ -n "${HADOOP_HOME}" ] && [ -d "${HADOOP_HOME}/lib/native/" ]; then
+   if [[ "${JVM_OPTS}${JVM_XOPTS}" != *-Djava.library.path=* ]]; then
+      JVM_OPTS="${JVM_OPTS} -Djava.library.path=${HADOOP_HOME}/lib/native/"
+   fi
+fi
+
+#
 # Set main class to start service (grid node by default).
 #
 if [ "${MAIN_CLASS}" = "" ]; then


[10/16] ignite git commit: IGNITE-4359 .NET: Support DateTime properties in LINQ

Posted by vo...@apache.org.
IGNITE-4359 .NET: Support DateTime properties in LINQ


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

Branch: refs/heads/ignite-2.0
Commit: 3bae858986f78dbb288cf39d970b3f163c3da247
Parents: 126ab60
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Dec 20 15:32:39 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Dec 20 15:32:39 2016 +0300

----------------------------------------------------------------------
 .../Cache/Query/CacheLinqTest.cs                | 24 ++++++++++----
 .../Impl/CacheQueryExpressionVisitor.cs         | 12 ++-----
 .../Apache.Ignite.Linq/Impl/MethodVisitor.cs    | 34 ++++++++++++++++++--
 3 files changed, 52 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3bae8589/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
index 1ac7fa7..46979ef 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
@@ -63,7 +63,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         private bool _runDbConsole;
 
         /** */
-        private static readonly DateTime StartDateTime = new DateTime(2000, 1, 1, 0, 0, 0, DateTimeKind.Utc);
+        private static readonly DateTime StartDateTime = new DateTime(2000, 5, 17, 15, 4, 5, DateTimeKind.Utc);
 
         /// <summary>
         /// Fixture set up.
@@ -111,7 +111,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
 
             roleCache[new RoleKey(1, 101)] = new Role {Name = "Role_1", Date = StartDateTime};
             roleCache[new RoleKey(2, 102)] = new Role {Name = "Role_2", Date = StartDateTime.AddYears(1)};
-            roleCache[new RoleKey(3, 103)] = new Role {Name = null, Date = StartDateTime.AddYears(2)};
+            roleCache[new RoleKey(3, 103)] = new Role {Name = null, Date = StartDateTime.AddHours(5432)};
         }
 
         /// <summary>
@@ -123,7 +123,8 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             {
                 BinaryConfiguration = new BinaryConfiguration(typeof(Person),
                     typeof(Organization), typeof(Address), typeof(Role), typeof(RoleKey), typeof(Numerics)),
-                GridName = gridName
+                GridName = gridName,
+                JvmOptions = { "-Duser.timezone=UTC" }
             };
         }
 
@@ -735,7 +736,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
 
             // Test retrieval
             var dates = roles.OrderBy(x => x.Value.Date).Select(x => x.Value.Date);
-            var expDates = new[] {StartDateTime, StartDateTime.AddYears(1), StartDateTime.AddYears(2)};
+            var expDates = GetRoleCache().Select(x => x.Value.Date).OrderBy(x => x).ToArray();
             Assert.AreEqual(expDates, dates.ToArray());
 
             // Filtering
@@ -748,10 +749,21 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
                 join person in persons on role.Value.Date equals person.Value.Birthday
                 select person;
 
-            Assert.AreEqual(RoleCount, join.Count());
+            Assert.AreEqual(2, join.Count());
 
             // Functions
-            Assert.AreEqual("01 01 2000 00:00:00", dates.Select(x => x.ToString("DD MM YYYY HH:mm:ss")).First());
+            var strings = dates.Select(x => x.ToString("dd MM YYYY HH:mm:ss")).ToArray();
+            Assert.AreEqual(new[] {"17 05 2000 15:04:05", "29 12 2000 23:04:05", "17 05 2001 15:04:05"}, strings);
+
+            // Properties
+            Assert.AreEqual(new[] {2000, 2000, 2001}, dates.Select(x => x.Year).ToArray());
+            Assert.AreEqual(new[] {5, 12, 5}, dates.Select(x => x.Month).ToArray());
+            Assert.AreEqual(new[] {17, 29, 17}, dates.Select(x => x.Day).ToArray());
+            Assert.AreEqual(expDates.Select(x => x.DayOfYear).ToArray(), dates.Select(x => x.DayOfYear).ToArray());
+            Assert.AreEqual(expDates.Select(x => x.DayOfWeek).ToArray(), dates.Select(x => x.DayOfWeek).ToArray());
+            Assert.AreEqual(new[] {15, 23, 15}, dates.Select(x => x.Hour).ToArray());
+            Assert.AreEqual(new[] { 4, 4, 4 }, dates.Select(x => x.Minute).ToArray());
+            Assert.AreEqual(new[] { 5, 5, 5 }, dates.Select(x => x.Second).ToArray());
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bae8589/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
index 8231053..1f9da1c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
@@ -276,17 +276,9 @@ namespace Apache.Ignite.Linq.Impl
         {
             // Field hierarchy is flattened (Person.Address.Street is just Street), append as is, do not call Visit.
 
-            // Special case: string.Length
-            if (expression.Member == MethodVisitor.StringLength)
-            {
-                ResultBuilder.Append("length(");
-
-                VisitMember((MemberExpression) expression.Expression);
-
-                ResultBuilder.Append(")");
-
+            // Property call (string.Length, DateTime.Month, etc).
+            if (MethodVisitor.VisitPropertyCall(expression, this))
                 return expression;
-            }
 
             // Special case: grouping
             if (VisitGroupByMember(expression.Expression))

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bae8589/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
index 3c8acc7..e83c448 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
@@ -29,8 +29,19 @@ namespace Apache.Ignite.Linq.Impl
     /// </summary>
     internal static class MethodVisitor
     {
-        /// <summary> The string length method. </summary>
-        public static readonly MemberInfo StringLength = typeof (string).GetProperty("Length");
+        /// <summary> Property visitors. </summary>
+        private static readonly Dictionary<MemberInfo, string> Properties = new Dictionary<MemberInfo, string>
+        {
+            {typeof(string).GetProperty("Length"), "length"},
+            {typeof(DateTime).GetProperty("Year"), "year"},
+            {typeof(DateTime).GetProperty("Month"), "month"},
+            {typeof(DateTime).GetProperty("Day"), "day_of_month"},
+            {typeof(DateTime).GetProperty("DayOfYear"), "day_of_year"},
+            {typeof(DateTime).GetProperty("DayOfWeek"), "-1 + day_of_week"},
+            {typeof(DateTime).GetProperty("Hour"), "hour"},
+            {typeof(DateTime).GetProperty("Minute"), "minute"},
+            {typeof(DateTime).GetProperty("Second"), "second"}
+        };
 
         /// <summary> Method visit delegate. </summary>
         private delegate void VisitMethodDelegate(MethodCallExpression expression, CacheQueryExpressionVisitor visitor);
@@ -103,6 +114,25 @@ namespace Apache.Ignite.Linq.Impl
         }.ToDictionary(x => x.Key, x => x.Value);
 
         /// <summary>
+        /// Visits the property call expression.
+        /// </summary>
+        public static bool VisitPropertyCall(MemberExpression expression, CacheQueryExpressionVisitor visitor)
+        {
+            string funcName;
+
+            if (!Properties.TryGetValue(expression.Member, out funcName))
+                return false;
+
+            visitor.ResultBuilder.Append(funcName).Append('(');
+
+            visitor.Visit(expression.Expression);
+
+            visitor.ResultBuilder.Append(')');
+
+            return true;
+        }
+
+        /// <summary>
         /// Visits the method call expression.
         /// </summary>
         public static void VisitMethodCall(MethodCallExpression expression, CacheQueryExpressionVisitor visitor)


[04/16] ignite git commit: IGNITE-3514 C++: Fixed JVM crash on incorrect configuration

Posted by vo...@apache.org.
IGNITE-3514 C++: Fixed JVM crash on incorrect configuration

This closes #1354


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

Branch: refs/heads/ignite-2.0
Commit: a38ae7eca125c6a17a468f130502de69e2aac18b
Parents: c530d47
Author: Igor Sapego <is...@gridgain.com>
Authored: Fri Dec 16 19:42:06 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Dec 16 19:42:06 2016 +0300

----------------------------------------------------------------------
 .../cpp/core-test/config/cache-test.xml         |  1 +
 .../platforms/cpp/core-test/config/invalid.xml  | 39 ++++++++++++++++++++
 .../cpp/core-test/project/vs/core-test.vcxproj  |  3 +-
 .../project/vs/core-test.vcxproj.filters        |  5 ++-
 .../cpp/core-test/src/interop_test.cpp          | 17 +++++++--
 modules/platforms/cpp/core/src/ignition.cpp     | 15 ++------
 6 files changed, 64 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a38ae7ec/modules/platforms/cpp/core-test/config/cache-test.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/config/cache-test.xml b/modules/platforms/cpp/core-test/config/cache-test.xml
index 659e12d..0ea5876 100644
--- a/modules/platforms/cpp/core-test/config/cache-test.xml
+++ b/modules/platforms/cpp/core-test/config/cache-test.xml
@@ -115,6 +115,7 @@
                         </property>
                     </bean>
                 </property>
+                <property name="socketTimeout" value="300" />
             </bean>
         </property>
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a38ae7ec/modules/platforms/cpp/core-test/config/invalid.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/config/invalid.xml b/modules/platforms/cpp/core-test/config/invalid.xml
new file mode 100644
index 0000000..d6481c7
--- /dev/null
+++ b/modules/platforms/cpp/core-test/config/invalid.xml
@@ -0,0 +1,39 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+    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.
+-->
+
+<!--
+    Ignite Spring configuration file to startup grid cache.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="socketTimeout" value="-300" />
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/a38ae7ec/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
index c18b1c3..6f13719 100644
--- a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
+++ b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
@@ -35,6 +35,7 @@
   <ItemGroup>
     <None Include="..\..\config\cache-query.xml" />
     <None Include="..\..\config\cache-test.xml" />
+    <None Include="..\..\config\invalid.xml" />
   </ItemGroup>
   <ItemGroup>
     <ClCompile Include="..\..\src\cache_test.cpp" />
@@ -185,4 +186,4 @@
   <Import Project="$(VCTargetsPath)\Microsoft.Cpp.targets" />
   <ImportGroup Label="ExtensionTargets">
   </ImportGroup>
-</Project>
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a38ae7ec/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
index 0059364..cf1aaca 100644
--- a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
+++ b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
@@ -88,5 +88,8 @@
     <None Include="..\..\config\cache-query.xml">
       <Filter>Configs</Filter>
     </None>
+    <None Include="..\..\config\invalid.xml">
+      <Filter>Configs</Filter>
+    </None>
   </ItemGroup>
-</Project>
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a38ae7ec/modules/platforms/cpp/core-test/src/interop_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/interop_test.cpp b/modules/platforms/cpp/core-test/src/interop_test.cpp
index 4b079d9..f29ecc8 100644
--- a/modules/platforms/cpp/core-test/src/interop_test.cpp
+++ b/modules/platforms/cpp/core-test/src/interop_test.cpp
@@ -27,7 +27,7 @@ using namespace ignite;
 using namespace cache;
 using namespace boost::unit_test;
 
-void InitConfig(IgniteConfiguration& cfg)
+void InitConfig(IgniteConfiguration& cfg, const char* config)
 {
     cfg.jvmOpts.push_back("-Xdebug");
     cfg.jvmOpts.push_back("-Xnoagent");
@@ -45,7 +45,7 @@ void InitConfig(IgniteConfiguration& cfg)
 
     char* cfgPath = getenv("IGNITE_NATIVE_TEST_CPP_CONFIG_PATH");
 
-    cfg.springCfgPath = std::string(cfgPath).append("/").append("cache-test.xml");
+    cfg.springCfgPath = std::string(cfgPath).append("/").append(config);
 }
 
 BOOST_AUTO_TEST_SUITE(InteropTestSuite)
@@ -119,7 +119,7 @@ BOOST_AUTO_TEST_CASE(StringUtfValid4ByteCodePoint)
 {
     IgniteConfiguration cfg;
 
-    InitConfig(cfg);
+    InitConfig(cfg, "cache-test.xml");
 
     Ignite ignite = Ignition::Start(cfg);
 
@@ -145,4 +145,15 @@ BOOST_AUTO_TEST_CASE(StringUtfValid4ByteCodePoint)
     Ignition::StopAll(false);
 }
 
+BOOST_AUTO_TEST_CASE(GracefulDeathOnInvalidConfig)
+{
+    IgniteConfiguration cfg;
+
+    InitConfig(cfg, "invalid.xml");
+
+    BOOST_CHECK_THROW(Ignition::Start(cfg), IgniteError);
+
+    Ignition::StopAll(false);
+}
+
 BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/a38ae7ec/modules/platforms/cpp/core/src/ignition.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/ignition.cpp b/modules/platforms/cpp/core/src/ignition.cpp
index 72fbf7a..cd7ad70 100644
--- a/modules/platforms/cpp/core/src/ignition.cpp
+++ b/modules/platforms/cpp/core/src/ignition.cpp
@@ -282,20 +282,13 @@ namespace ignite
         factoryLock.Leave();
 
         if (failed) 
-        {
-            if (envTarget)
-                delete envTarget;
-
             return Ignite();
-        }
-        else 
-        {
-            env.Get()->ProcessorReleaseStart();
 
-            IgniteImpl* impl = new IgniteImpl(env, javaRef);
+        env.Get()->ProcessorReleaseStart();
 
-            return Ignite(impl);
-        }
+        IgniteImpl* impl = new IgniteImpl(env, javaRef);
+
+        return Ignite(impl);
     }
 
     Ignite Ignition::Get()


[02/16] ignite git commit: ignite-2412 Do not call 'asyncOp' for synchronous operations

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/c530d47b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
index a8219b0..4350b3e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
@@ -450,61 +450,11 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public V getAndPutIfAbsent(K key, V val) throws IgniteCheckedException {
-        return dht.getAndPutIfAbsent(key, val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<V> getAndPutIfAbsentAsync(K key, V val) {
-        return dht.getAndPutIfAbsentAsync(key, val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean putIfAbsent(K key, V val) throws IgniteCheckedException {
-        return dht.putIfAbsent(key, val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> putIfAbsentAsync(K key, V val) {
-        return dht.putIfAbsentAsync(key, val);
-    }
-
-    /** {@inheritDoc} */
     @Nullable @Override public V tryGetAndPut(K key, V val) throws IgniteCheckedException {
         return dht.tryGetAndPut(key, val);
     }
 
     /** {@inheritDoc} */
-    @Override public V getAndReplace(K key, V val) throws IgniteCheckedException {
-        return dht.getAndReplace(key, val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<V> getAndReplaceAsync(K key, V val) {
-        return dht.getAndReplaceAsync(key, val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean replace(K key, V val) throws IgniteCheckedException {
-        return dht.replace(key, val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> replaceAsync(K key, V val) {
-        return dht.replaceAsync(key, val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean replace(K key, V oldVal, V newVal) throws IgniteCheckedException {
-        return dht.replace(key, oldVal, newVal);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
-        return dht.replaceAsync(key, oldVal, newVal);
-    }
-
-    /** {@inheritDoc} */
     @Override public void putAll(Map<? extends K, ? extends V> m)
         throws IgniteCheckedException {
         dht.putAll(m);
@@ -569,6 +519,11 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean remove(K key, @Nullable CacheEntryPredicate filter) throws IgniteCheckedException {
+        return dht.remove(key, filter);
+    }
+
+    /** {@inheritDoc} */
     @Override public V getAndRemove(K key) throws IgniteCheckedException {
         return dht.getAndRemove(key);
     }
@@ -602,16 +557,6 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean remove(K key, V val) throws IgniteCheckedException {
-        return dht.remove(key, val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> removeAsync(K key, V val) {
-        return dht.removeAsync(key, val);
-    }
-
-    /** {@inheritDoc} */
     @Override public void removeAll() throws IgniteCheckedException {
         dht.removeAll();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c530d47b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
index bc16ff4..a26d2f3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -58,7 +58,6 @@ import org.apache.ignite.internal.processors.cache.local.GridLocalCache;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.resource.GridResourceIoc;
-import org.apache.ignite.internal.processors.resource.GridResourceProcessor;
 import org.apache.ignite.internal.util.F0;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.future.GridEmbeddedFuture;
@@ -108,6 +107,11 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
     }
 
     /** {@inheritDoc} */
+    @Override protected void checkJta() throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean isLocal() {
         return true;
     }
@@ -119,9 +123,7 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public V getAndPut(K key, V val, @Nullable CacheEntryPredicate filter) throws IgniteCheckedException {
-        A.notNull(key, "key", val, "val");
-
+    @Override protected V getAndPut0(K key, V val, @Nullable CacheEntryPredicate filter) throws IgniteCheckedException {
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
         return (V)updateAllInternal(UPDATE,
@@ -138,16 +140,10 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean put(K key, V val, CacheEntryPredicate filter) throws IgniteCheckedException {
-        A.notNull(key, "key", val, "val");
-
-        boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        long start = statsEnabled ? System.nanoTime() : 0L;
-
+    @Override protected boolean put0(K key, V val, CacheEntryPredicate filter) throws IgniteCheckedException {
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
-        boolean res = (Boolean)updateAllInternal(UPDATE,
+        Boolean res = (Boolean)updateAllInternal(UPDATE,
             Collections.singleton(key),
             Collections.singleton(val),
             null,
@@ -159,8 +155,7 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
             ctx.readThrough(),
             opCtx != null && opCtx.isKeepBinary());
 
-        if (statsEnabled)
-            metrics0().addPutTimeNanos(System.nanoTime() - start);
+        assert res != null;
 
         return res;
     }
@@ -168,8 +163,6 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public IgniteInternalFuture<V> getAndPutAsync0(K key, V val, @Nullable CacheEntryPredicate filter) {
-        A.notNull(key, "key", val, "val");
-
         return updateAllAsync0(F0.asMap(key, val),
             null,
             null,
@@ -181,8 +174,6 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public IgniteInternalFuture<Boolean> putAsync0(K key, V val, @Nullable CacheEntryPredicate filter) {
-        A.notNull(key, "key", val, "val");
-
         return updateAllAsync0(F0.asMap(key, val),
             null,
             null,
@@ -192,65 +183,7 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public V getAndPutIfAbsent(K key, V val) throws IgniteCheckedException {
-        return getAndPut(key, val, ctx.noVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<V> getAndPutIfAbsentAsync(K key, V val) {
-        return getAndPutAsync(key, val, ctx.noVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean putIfAbsent(K key, V val) throws IgniteCheckedException {
-        return put(key, val, ctx.noVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> putIfAbsentAsync(K key, V val) {
-        return putAsync(key, val, ctx.noVal());
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public V getAndReplace(K key, V val) throws IgniteCheckedException {
-        return getAndPut(key, val, ctx.hasVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<V> getAndReplaceAsync(K key, V val) {
-        return getAndPutAsync(key, val, ctx.hasVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean replace(K key, V val) throws IgniteCheckedException {
-        return put(key, val, ctx.hasVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> replaceAsync(K key, V val) {
-        return putAsync(key, val, ctx.hasVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean replace(K key, V oldVal, V newVal) throws IgniteCheckedException {
-        A.notNull(oldVal, "oldVal");
-
-        return put(key, newVal, ctx.equalsVal(oldVal));
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
-        return putAsync(key, newVal, ctx.equalsVal(oldVal));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void putAll(Map<? extends K, ? extends V> m) throws IgniteCheckedException {
-        boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        long start = statsEnabled ? System.nanoTime() : 0L;
-
+    @Override protected void putAll0(Map<? extends K, ? extends V> m) throws IgniteCheckedException {
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
         updateAllInternal(UPDATE,
@@ -264,13 +197,10 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
             ctx.writeThrough(),
             ctx.readThrough(),
             opCtx != null && opCtx.isKeepBinary());
-
-        if (statsEnabled)
-            metrics0().addPutTimeNanos(System.nanoTime() - start);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> putAllAsync(Map<? extends K, ? extends V> m) {
+    @Override public IgniteInternalFuture<?> putAllAsync0(Map<? extends K, ? extends V> m) {
         return updateAllAsync0(m,
             null,
             null,
@@ -280,8 +210,7 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public V getAndRemove(K key) throws IgniteCheckedException {
+    @Override protected V getAndRemove0(K key) throws IgniteCheckedException {
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
         return (V)updateAllInternal(DELETE,
@@ -299,13 +228,13 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteInternalFuture<V> getAndRemoveAsync(K key) {
+    @Override public IgniteInternalFuture<V> getAndRemoveAsync0(K key) {
         return removeAllAsync0(Collections.singletonList(key), true, false, null);
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public void removeAll(Collection<? extends K> keys) throws IgniteCheckedException {
+    @Override public void removeAll0(Collection<? extends K> keys) throws IgniteCheckedException {
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
         updateAllInternal(DELETE,
@@ -322,19 +251,13 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> removeAllAsync(Collection<? extends K> keys) {
+    @Override public IgniteInternalFuture<Object> removeAllAsync0(Collection<? extends K> keys) {
         return removeAllAsync0(keys, false, false, null).chain(RET2NULL);
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public boolean remove(K key) throws IgniteCheckedException {
-        boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        long start = statsEnabled ? System.nanoTime() : 0L;
-
-        A.notNull(key, "key");
-
+    @Override public boolean remove0(K key, final CacheEntryPredicate filter) throws IgniteCheckedException {
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
         Boolean rmv = (Boolean)updateAllInternal(DELETE,
@@ -344,50 +267,23 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
             expiryPerCall(),
             false,
             false,
-            null,
+            filter,
             ctx.writeThrough(),
             ctx.readThrough(),
             opCtx != null && opCtx.isKeepBinary());
 
-        if (statsEnabled && rmv)
-            metrics0().addRemoveTimeNanos(System.nanoTime() - start);
+        assert rmv != null;
 
         return rmv;
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteInternalFuture<Boolean> removeAsync(K key, @Nullable CacheEntryPredicate filter) {
-        A.notNull(key, "key");
-
+    @Override public IgniteInternalFuture<Boolean> removeAsync0(K key, @Nullable CacheEntryPredicate filter) {
         return removeAllAsync0(Collections.singletonList(key), false, false, filter);
     }
 
     /** {@inheritDoc} */
-    @Override public boolean remove(K key, V val) throws IgniteCheckedException {
-        A.notNull(key, "key", val, "val");
-
-        CacheOperationContext opCtx = ctx.operationContextPerCall();
-
-        return (Boolean)updateAllInternal(DELETE,
-            Collections.singleton(key),
-            null,
-            null,
-            expiryPerCall(),
-            false,
-            false,
-            ctx.equalsVal(val),
-            ctx.writeThrough(),
-            ctx.readThrough(),
-            opCtx != null && opCtx.isKeepBinary());
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> removeAsync(K key, V val) {
-        return removeAsync(key, ctx.equalsVal(val));
-    }
-
-    /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> removeAllAsync() {
         return ctx.closures().callLocalSafe(new Callable<Void>() {
             @Override public Void call() throws Exception {
@@ -399,11 +295,13 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
     }
 
     /** {@inheritDoc} */
-
     @SuppressWarnings("unchecked")
-    @Override @Nullable public V get(K key, boolean deserializeBinary, boolean needVer) throws IgniteCheckedException {
-        String taskName = ctx.kernalContext().job().currentTaskName();
-
+    @Override protected V get0(
+        final K key,
+        String taskName,
+        boolean deserializeBinary,
+        boolean needVer) throws IgniteCheckedException
+    {
         Map<K, V> m = getAllInternal(Collections.singleton(key),
             ctx.isSwapOrOffheapEnabled(),
             ctx.readThrough(),
@@ -419,7 +317,7 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public final Map<K, V> getAll(Collection<? extends K> keys, boolean deserializeBinary, boolean needVer)
+    @Override public final Map<K, V> getAll0(Collection<? extends K> keys, boolean deserializeBinary, boolean needVer)
         throws IgniteCheckedException {
         A.notNull(keys, "keys");
 
@@ -794,7 +692,7 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
 
         final boolean keepBinary = opCtx != null && opCtx.isKeepBinary();
 
-        IgniteInternalFuture fut = asyncOp(new Callable<Object>() {
+        return asyncOp(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 return updateAllInternal(op,
                     keys,
@@ -809,11 +707,6 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
                     keepBinary);
             }
         });
-
-        if (ctx.config().isStatisticsEnabled())
-            fut.listen(new UpdatePutTimeStatClosure(metrics0(), System.nanoTime()));
-
-        return fut;
     }
 
     /**
@@ -835,17 +728,13 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
 
         final boolean readThrough = ctx.readThrough();
 
-        final boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
         final ExpiryPolicy expiryPlc = expiryPerCall();
 
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
         final boolean keepBinary = opCtx != null && opCtx.isKeepBinary();
 
-        IgniteInternalFuture fut = asyncOp(new Callable<Object>() {
+        return asyncOp(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 return updateAllInternal(DELETE,
                     keys,
@@ -860,11 +749,6 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
                     keepBinary);
             }
         });
-
-        if (statsEnabled)
-            fut.listen(new UpdateRemoveTimeStatClosure<>(metrics0(), start));
-
-        return fut;
     }
 
     /**
@@ -1584,10 +1468,7 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
      * @return Future.
      */
     @SuppressWarnings("unchecked")
-    protected IgniteInternalFuture asyncOp(final Callable<?> op) {
-        if (!asyncToggled)
-            return ctx.closures().callLocalSafe(op);
-
+    private IgniteInternalFuture asyncOp(final Callable<?> op) {
         IgniteInternalFuture fail = asyncOpAcquire();
 
         if (fail != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/c530d47b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.java
index 3e3b84e..648134e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.java
@@ -34,7 +34,7 @@ public class CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest extend
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        cfg.getTransactionConfiguration().setDefaultTxTimeout(Long.MAX_VALUE);
+        cfg.getTransactionConfiguration().setDefaultTxTimeout(5 * 60_000);
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c530d47b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBlockMessageSystemPoolStarvationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBlockMessageSystemPoolStarvationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBlockMessageSystemPoolStarvationSelfTest.java
index ec3b808..57c709b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBlockMessageSystemPoolStarvationSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBlockMessageSystemPoolStarvationSelfTest.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteFileSystem;
@@ -39,14 +41,11 @@ import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.transactions.TransactionConcurrency;
-import org.apache.ignite.transactions.TransactionIsolation;
-
-import java.util.concurrent.Callable;
-import java.util.concurrent.CountDownLatch;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 
 /**
  * Test to check for system pool starvation due to {@link IgfsBlocksMessage}.
@@ -125,8 +124,7 @@ public class IgfsBlockMessageSystemPoolStarvationSelfTest extends IgfsCommonAbst
             @Override public Void call() throws Exception {
                 GridCacheAdapter dataCache = dataCache(attacker);
 
-                try (IgniteInternalTx tx =
-                         dataCache.txStartEx(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
+                try (IgniteInternalTx tx = dataCache.txStartEx(PESSIMISTIC, REPEATABLE_READ)) {
                     dataCache.put(DATA_KEY, 0);
 
                     txStartLatch.countDown();
@@ -185,6 +183,7 @@ public class IgfsBlockMessageSystemPoolStarvationSelfTest extends IgfsCommonAbst
      * Create IGFS file asynchronously.
      *
      * @param path Path.
+     * @param writeStartLatch Write start latch.
      * @return Future.
      */
     private IgniteInternalFuture<Void> createFileAsync(final IgfsPath path, final CountDownLatch writeStartLatch) {
@@ -265,6 +264,7 @@ public class IgfsBlockMessageSystemPoolStarvationSelfTest extends IgfsCommonAbst
         cfg.setLocalHost("127.0.0.1");
         cfg.setConnectorConfiguration(null);
 
+        cfg.setStripedPoolSize(0);
         cfg.setSystemThreadPoolSize(2);
         cfg.setRebalanceThreadPoolSize(1);
         cfg.setPublicThreadPoolSize(1);


[08/16] ignite git commit: Null argument check for cache.putAll.

Posted by vo...@apache.org.
Null argument check for cache.putAll.


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

Branch: refs/heads/ignite-2.0
Commit: c0f1a219e0e423cb65f25a3968318495d29ab86f
Parents: cf7b6a0
Author: sboikov <sb...@gridgain.com>
Authored: Tue Dec 20 12:35:51 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Dec 20 12:35:51 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/cache/GridCacheAdapter.java  | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c0f1a219/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 5707b49..50869be 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -2664,6 +2664,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public void putAll(@Nullable final Map<? extends K, ? extends V> m) throws IgniteCheckedException {
+        A.notNull(m, "map");
+
         if (F.isEmpty(m))
             return;
 


[09/16] ignite git commit: Fixed 'singleRmv' flag for cache.remove(k, v) operation in tx cache.

Posted by vo...@apache.org.
Fixed 'singleRmv' flag for cache.remove(k, v) operation in tx cache.


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

Branch: refs/heads/ignite-2.0
Commit: 126ab60fe6fa0f47e19a26dafecc7feb7c57b60b
Parents: c0f1a21
Author: sboikov <sb...@gridgain.com>
Authored: Tue Dec 20 15:15:04 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Dec 20 15:15:04 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/cache/GridCacheAdapter.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/126ab60f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 50869be..74d1553 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -2951,6 +2951,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /**
      * @param key Key.
+     * @param filter Filter.
      * @return {@code True} if entry was removed.
      * @throws IgniteCheckedException If failed.
      */
@@ -2962,7 +2963,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                     Collections.singletonList(key),
                     /*retval*/false,
                     filter,
-                    /*singleRmv*/true).get().success();
+                    /*singleRmv*/filter == null).get().success();
             }
 
             @Override public String toString() {


[11/16] ignite git commit: IGNITE-4359 .NET: Fix Java timezone setting in tests

Posted by vo...@apache.org.
IGNITE-4359 .NET: Fix Java timezone setting in tests


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

Branch: refs/heads/ignite-2.0
Commit: 700529a6e461d00362666de9de58a92cf7bf8bbe
Parents: 3bae858
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Dec 20 16:01:22 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Dec 20 16:01:22 2016 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs  | 3 +--
 modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs    | 3 ++-
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/700529a6/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
index 46979ef..e74f09f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
@@ -123,8 +123,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             {
                 BinaryConfiguration = new BinaryConfiguration(typeof(Person),
                     typeof(Organization), typeof(Address), typeof(Role), typeof(RoleKey), typeof(Numerics)),
-                GridName = gridName,
-                JvmOptions = { "-Duser.timezone=UTC" }
+                GridName = gridName
             };
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/700529a6/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
index 7d483bb..ad27398 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
@@ -55,7 +55,8 @@ namespace Apache.Ignite.Core.Tests
                 "-Xms1g",
                 "-Xmx4g",
                 "-ea",
-                "-DIGNITE_QUIET=true"
+                "-DIGNITE_QUIET=true",
+                "-Duser.timezone=UTC"
             }
             : new List<string>
             {


[13/16] ignite git commit: IGNITE-1443: Implemented ContinuousQuery for C++

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/cache/cache.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/cache.h b/modules/platforms/cpp/core/include/ignite/cache/cache.h
index a975be3..54c0f96 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/cache.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/cache.h
@@ -37,6 +37,8 @@
 #include "ignite/cache/query/query_sql.h"
 #include "ignite/cache/query/query_text.h"
 #include "ignite/cache/query/query_sql_fields.h"
+#include "ignite/cache/query/continuous/continuous_query_handle.h"
+#include "ignite/cache/query/continuous/continuous_query.h"
 #include "ignite/impl/cache/cache_impl.h"
 #include "ignite/impl/operations.h"
 
@@ -1339,6 +1341,106 @@ namespace ignite
             }
 
             /**
+             * Start continuous query execution.
+             *
+             * @param qry Continuous query.
+             * @return Continuous query handle.
+             */
+            query::continuous::ContinuousQueryHandle<K, V> QueryContinuous(
+                const query::continuous::ContinuousQuery<K, V>& qry)
+            {
+                IgniteError err;
+
+                query::continuous::ContinuousQueryHandle<K, V> res = QueryContinuous(qry, err);
+
+                IgniteError::ThrowIfNeeded(err);
+
+                return res;
+            }
+
+            /**
+             * Start continuous query execution.
+             *
+             * @param qry Continuous query.
+             * @param err Error.
+             * @return Continuous query handle.
+             */
+            query::continuous::ContinuousQueryHandle<K, V> QueryContinuous(
+                const query::continuous::ContinuousQuery<K, V>& qry, IgniteError& err)
+            {
+                using namespace impl::cache::query::continuous;
+
+                if (!qry.impl.IsValid() || !qry.impl.Get()->HasListener())
+                {
+                    err = IgniteError(IgniteError::IGNITE_ERR_GENERIC,
+                        "Event listener is not set for ContinuousQuery instance");
+
+                    return query::continuous::ContinuousQueryHandle<K, V>();
+                }
+
+                ContinuousQueryHandleImpl* cqImpl;
+                cqImpl = impl.Get()->QueryContinuous(qry.impl, err);
+
+                if (cqImpl)
+                    cqImpl->SetQuery(qry.impl);
+
+                return query::continuous::ContinuousQueryHandle<K, V>(cqImpl);
+            }
+
+            /**
+             * Start continuous query execution with the initial query.
+             *
+             * @param qry Continuous query.
+             * @param initialQry Initial query to be executed.
+             * @return Continuous query handle.
+             */
+            template<typename Q>
+            query::continuous::ContinuousQueryHandle<K, V> QueryContinuous(
+                const query::continuous::ContinuousQuery<K, V>& qry,
+                const Q& initialQry)
+            {
+                IgniteError err;
+
+                query::continuous::ContinuousQueryHandle<K, V> res = QueryContinuous(qry, initialQry, err);
+
+                IgniteError::ThrowIfNeeded(err);
+
+                return res;
+            }
+
+            /**
+             * Start continuous query execution with the initial query.
+             *
+             * @param qry Continuous query.
+             * @param initialQry Initial query to be executed.
+             * @param err Error.
+             * @return Continuous query handle.
+             */
+            template<typename Q>
+            query::continuous::ContinuousQueryHandle<K, V> QueryContinuous(
+                const query::continuous::ContinuousQuery<K, V>& qry,
+                const Q& initialQry, IgniteError& err)
+            {
+                using namespace impl::cache::query::continuous;
+
+                if (!qry.impl.IsValid() || !qry.impl.Get()->HasListener())
+                {
+                    err = IgniteError(IgniteError::IGNITE_ERR_GENERIC,
+                        "Event listener is not set for ContinuousQuery instance");
+
+                    return query::continuous::ContinuousQueryHandle<K, V>();
+                }
+
+                ContinuousQueryHandleImpl* cqImpl;
+                cqImpl = impl.Get()->QueryContinuous(qry.impl, initialQry, err);
+
+                if (cqImpl)
+                    cqImpl->SetQuery(qry.impl);
+
+                return query::continuous::ContinuousQueryHandle<K, V>(cqImpl);
+            }
+
+            /**
              * Check if the instance is valid.
              *
              * Invalid instance can be returned if some of the previous
@@ -1356,7 +1458,7 @@ namespace ignite
 
         private:
             /** Implementation delegate. */
-            ignite::common::concurrent::SharedPointer<impl::cache::CacheImpl> impl;
+            common::concurrent::SharedPointer<impl::cache::CacheImpl> impl;
         };
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/cache/cache_entry.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/cache_entry.h b/modules/platforms/cpp/core/include/ignite/cache/cache_entry.h
index c737940..aea5182 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/cache_entry.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/cache_entry.h
@@ -45,7 +45,9 @@ namespace ignite
              * Creates instance with both key and value default-constructed.
              */
             CacheEntry() :
-                key(), val()
+                key(),
+                val(),
+                hasValue(false)
             {
                 // No-op.
             }
@@ -57,7 +59,9 @@ namespace ignite
              * @param val Value.
              */
             CacheEntry(const K& key, const V& val) :
-                key(key), val(val)
+                key(key),
+                val(val),
+                hasValue(true)
             {
                 // No-op.
             }
@@ -68,7 +72,17 @@ namespace ignite
              * @param other Other instance.
              */
             CacheEntry(const CacheEntry& other) :
-                key(other.key), val(other.val)
+                key(other.key),
+                val(other.val),
+                hasValue(other.hasValue)
+            {
+                // No-op.
+            }
+
+            /**
+             * Destructor.
+             */
+            virtual ~CacheEntry()
             {
                 // No-op.
             }
@@ -84,6 +98,7 @@ namespace ignite
                 {
                     key = other.key;
                     val = other.val;
+                    hasValue = other.hasValue;
                 }
 
                 return *this;
@@ -94,7 +109,7 @@ namespace ignite
              *
              * @return Key.
              */
-            K GetKey() const
+            const K& GetKey() const
             {
                 return key;
             }
@@ -104,17 +119,30 @@ namespace ignite
              *
              * @return Value.
              */
-            V GetValue() const
+            const V& GetValue() const
             {
                 return val;
             }
 
-        private:
+            /**
+             * Check if the value exists.
+             *
+             * @return True, if the value exists.
+             */
+            bool HasValue() const
+            {
+                return hasValue;
+            }
+
+        protected:
             /** Key. */
             K key;
 
             /** Value. */
             V val;
+
+            /** Indicates whether value exists */
+            bool hasValue;
         };
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/cache/event/cache_entry_event.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/event/cache_entry_event.h b/modules/platforms/cpp/core/include/ignite/cache/event/cache_entry_event.h
new file mode 100644
index 0000000..14fa185
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/cache/event/cache_entry_event.h
@@ -0,0 +1,139 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::cache::event::CacheEntryEvent class.
+ */
+
+#ifndef _IGNITE_CACHE_EVENT_CACHE_ENTRY_EVENT
+#define _IGNITE_CACHE_EVENT_CACHE_ENTRY_EVENT
+
+#include <ignite/binary/binary_raw_reader.h>
+#include <ignite/cache/cache_entry.h>
+
+namespace ignite
+{
+    namespace cache
+    {
+        /**
+         * Cache entry event class template.
+         *
+         * Both key and value types should be default-constructable,
+         * copy-constructable and assignable.
+         */
+        template<typename K, typename V>
+        class CacheEntryEvent : public CacheEntry<K, V>
+        {
+        public:
+            /**
+             * Default constructor.
+             *
+             * Creates instance with all fields default-constructed.
+             */
+            CacheEntryEvent() :
+                CacheEntry<K, V>(),
+                oldVal(),
+                hasOldValue(false)
+            {
+                // No-op.
+            }
+
+            /**
+             * Copy constructor.
+             *
+             * @param other Other instance.
+             */
+            CacheEntryEvent(const CacheEntryEvent<K, V>& other) :
+                CacheEntry<K, V>(other),
+                oldVal(other.oldVal),
+                hasOldValue(other.hasOldValue)
+            {
+                // No-op.
+            }
+
+            /**
+             * Destructor.
+             */
+            virtual ~CacheEntryEvent()
+            {
+                // No-op.
+            }
+
+            /**
+             * Assignment operator.
+             *
+             * @param other Other instance.
+             * @return *this.
+             */
+            CacheEntryEvent& operator=(const CacheEntryEvent<K, V>& other)
+            {
+                if (this != &other)
+                {
+                    CacheEntry<K, V>::operator=(other);
+
+                    oldVal = other.oldVal;
+                    hasOldValue = other.hasOldValue;
+                }
+
+                return *this;
+            }
+
+            /**
+             * Get old value.
+             *
+             * @return Old value.
+             */
+            const V& GetOldValue() const
+            {
+                return oldVal;
+            }
+
+            /**
+             * Check if the old value exists.
+             *
+             * @return True, if the old value exists.
+             */
+            bool HasOldValue() const
+            {
+                return hasOldValue;
+            }
+
+            /**
+             * Reads cache event using provided raw reader.
+             *
+             * @param reader Reader to use.
+             */
+            void Read(binary::BinaryRawReader& reader)
+            {
+                this->key = reader.ReadObject<K>();
+
+                this->hasOldValue = reader.TryReadObject(this->oldVal);
+                this->hasValue = reader.TryReadObject(this->val);
+            }
+
+        private:
+            /** Old value. */
+            V oldVal;
+
+            /** Indicates whether old value exists */
+            bool hasOldValue;
+        };
+    }
+}
+
+#endif //_IGNITE_CACHE_EVENT_CACHE_ENTRY_EVENT

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/cache/event/cache_entry_event_listener.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/event/cache_entry_event_listener.h b/modules/platforms/cpp/core/include/ignite/cache/event/cache_entry_event_listener.h
new file mode 100644
index 0000000..dd8f4a2
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/cache/event/cache_entry_event_listener.h
@@ -0,0 +1,71 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::cache::event::CacheEntryEventListener class.
+ */
+
+#ifndef _IGNITE_CACHE_EVENT_CACHE_ENTRY_EVENT_LISTENER
+#define _IGNITE_CACHE_EVENT_CACHE_ENTRY_EVENT_LISTENER
+
+#include <stdint.h>
+
+#include <ignite/cache/event/cache_entry_event.h>
+
+namespace ignite
+{
+    namespace cache
+    {
+        namespace event
+        {
+            /**
+             * Cache entry event listener.
+             */
+            template<typename K, typename V>
+            class CacheEntryEventListener
+            {
+            public:
+                /**
+                 * Default constructor.
+                 */
+                CacheEntryEventListener()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Destructor.
+                 */
+                virtual ~CacheEntryEventListener()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Event callback.
+                 *
+                 * @param evts Events.
+                 * @param num Events number.
+                 */
+                virtual void OnEvent(const CacheEntryEvent<K, V>* evts, uint32_t num) = 0;
+            };
+        }
+    }
+}
+
+#endif //_IGNITE_CACHE_EVENT_CACHE_ENTRY_EVENT_LISTENER
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h b/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h
new file mode 100644
index 0000000..563b11a
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h
@@ -0,0 +1,239 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::cache::query::continuous::ContinuousQuery class.
+ */
+
+#ifndef _IGNITE_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY
+#define _IGNITE_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY
+
+#include <ignite/impl/cache/query/continuous/continuous_query_impl.h>
+#include <ignite/cache/event/cache_entry_event_listener.h>
+
+namespace ignite
+{
+    namespace cache
+    {
+        // Forward-declaration.
+        template<typename K, typename V>
+        class IGNITE_IMPORT_EXPORT Cache;
+
+        namespace query
+        {
+            namespace continuous
+            {
+                /**
+                 * Continuous query.
+                 *
+                 * Continuous queries allow to register a remote and a listener
+                 * for cache update events. On any update to the related cache
+                 * an event is sent to the node that has executed the query and
+                 * listener is notified on that node.
+                 *
+                 * Continuous query can either be executed on the whole topology
+                 * or only on local node.
+                 *
+                 * To execute the query over the cache use method
+                 * ignite::cache::Cache::QueryContinuous().
+                 */
+                template<typename K, typename V>
+                class ContinuousQuery
+                {
+                    friend class Cache<K, V>;
+                public:
+
+                    /**
+                     * Default value for the buffer size.
+                     */
+                    enum { DEFAULT_BUFFER_SIZE = 1 };
+
+                    /**
+                     * Default value for the time interval.
+                     */
+                    enum { DEFAULT_TIME_INTERVAL = 0 };
+
+                    /**
+                     * Destructor.
+                     */
+                    ~ContinuousQuery()
+                    {
+                        // No-op.
+                    }
+
+                    /**
+                     * Constructor.
+                     *
+                     * @param lsnr Event listener. Invoked on the node where
+                     *     continuous query execution has been started.
+                     */
+                    ContinuousQuery(Reference<event::CacheEntryEventListener<K, V>> lsnr) :
+                        impl(new impl::cache::query::continuous::ContinuousQueryImpl<K, V>(lsnr))
+                    {
+                        // No-op.
+                    }
+
+                    /**
+                     * Constructor.
+                     *
+                     * @param lsnr Event listener Invoked on the node where
+                     *     continuous query execution has been started.
+                     * @param loc Whether query should be executed locally.
+                     */
+                    ContinuousQuery(Reference<event::CacheEntryEventListener<K, V>> lsnr, bool loc) :
+                        impl(new impl::cache::query::continuous::ContinuousQueryImpl<K, V>(lsnr, loc))
+                    {
+                        // No-op.
+                    }
+
+                    /**
+                     * Set local flag.
+                     *
+                     * @param val Value of the flag. If true, query will be
+                     *     executed only on local node, so only local entries
+                     *     will be returned as query result.
+                     */
+                    void SetLocal(bool val)
+                    {
+                        impl.Get()->SetLocal(val);
+                    }
+
+                    /**
+                     * Get local flag.
+                     *
+                     * @return Value of the flag. If true, query will be
+                     *     executed only on local node, so only local entries
+                     *     will be returned as query result.
+                     */
+                    bool GetLocal() const
+                    {
+                        return impl.Get()->GetLocal();
+                    }
+
+                    /**
+                     * Set buffer size.
+                     *
+                     * When a cache update happens, entry is first
+                     * put into a buffer. Entries from buffer will be sent to
+                     * the master node only if the buffer is full or time
+                     * provided via timeInterval is exceeded.
+                     *
+                     * @param val Buffer size.
+                     */
+                    void SetBufferSize(int32_t val)
+                    {
+                        impl.Get()->SetBufferSize(val);
+                    }
+
+                    /**
+                     * Get buffer size.
+                     *
+                     * When a cache update happens, entry is first
+                     * put into a buffer. Entries from buffer will be sent to
+                     * the master node only if the buffer is full or time
+                     * provided via timeInterval is exceeded.
+                     *
+                     * @return Buffer size.
+                     */
+                    int32_t GetBufferSize() const
+                    {
+                        return impl.Get()->GetBufferSize();
+                    }
+
+                    /**
+                     * Set time interval.
+                     *
+                     * When a cache update happens, entry is first put into
+                     * a buffer. Entries from buffer are sent to the master node
+                     * only if the buffer is full (its size can be changed via
+                     * SetBufferSize) or time provided via this method is
+                     * exceeded.
+                     *
+                     * Default value is DEFAULT_TIME_INTERVAL, i.e. 0, which
+                     * means that time check is disabled and entries will be
+                     * sent only when buffer is full.
+                     *
+                     * @param val Time interval in miliseconds.
+                     */
+                    void SetTimeInterval(int64_t val)
+                    {
+                        impl.Get()->SetTimeInterval(val);
+                    }
+
+                    /**
+                     * Get time interval.
+                     *
+                     * When a cache update happens, entry is first put into
+                     * a buffer. Entries from buffer are sent to the master node
+                     * only if the buffer is full (its size can be changed via
+                     * SetBufferSize) or time provided via SetTimeInterval
+                     * method is exceeded.
+                     *
+                     * Default value is DEFAULT_TIME_INTERVAL, i.e. 0, which
+                     * means that time check is disabled and entries will be
+                     * sent only when buffer is full.
+                     *
+                     * @return Time interval.
+                     */
+                    int64_t GetTimeInterval() const
+                    {
+                        return impl.Get()->GetTimeInterval();
+                    }
+
+                    /**
+                     * Set cache entry event listener.
+                     *
+                     * @param val Cache entry event listener. Invoked on the
+                     *     node where continuous query execution has been
+                     *     started.
+                     */
+                    void SetListener(Reference<event::CacheEntryEventListener<K, V>> lsnr)
+                    {
+                        impl.Get()->SetListener(val);
+                    }
+
+                    /**
+                     * Get cache entry event listener.
+                     *
+                     * @return Cache entry event listener.
+                     */
+                    const event::CacheEntryEventListener<K, V>& GetListener() const
+                    {
+                        return impl.Get()->GetListener();
+                    }
+
+                    /**
+                     * Get cache entry event listener.
+                     *
+                     * @return Cache entry event listener.
+                     */
+                    event::CacheEntryEventListener<K, V>& GetListener()
+                    {
+                        return impl.Get()->GetListener();
+                    }
+
+                private:
+                    /** Implementation. */
+                    common::concurrent::SharedPointer<impl::cache::query::continuous::ContinuousQueryImpl<K, V>> impl;
+                };
+            }
+        }
+    }
+}
+
+#endif //_IGNITE_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query_handle.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query_handle.h b/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query_handle.h
new file mode 100644
index 0000000..bbefbcc
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query_handle.h
@@ -0,0 +1,133 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::cache::query::continuous::ContinuousQueryHandle class.
+ */
+
+#ifndef _IGNITE_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY_HANDLE
+#define _IGNITE_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY_HANDLE
+
+#include <ignite/impl/cache/query/continuous/continuous_query_handle_impl.h>
+
+namespace ignite
+{
+    namespace cache
+    {
+        namespace query
+        {
+            namespace continuous
+            {
+                /**
+                 * Continuous query handle.
+                 */
+                template<typename K, typename V>
+                class ContinuousQueryHandle
+                {
+                public:
+                    /**
+                     * Default constructor.
+                     */
+                    ContinuousQueryHandle() :
+                        impl()
+                    {
+                        // No-op.
+                    }
+
+                    /**
+                     * Constructor.
+                     *
+                     * Internal method. Should not be used by user.
+                     *
+                     * @param impl Implementation.
+                     */
+                    ContinuousQueryHandle(impl::cache::query::continuous::ContinuousQueryHandleImpl* impl) :
+                        impl(impl)
+                    {
+                        // No-op.
+                    }
+
+                    /**
+                     * Gets the cursor for initial query.
+                     * Can be called only once, throws IgniteError on consequent
+                     * calls.
+                     *
+                     * @return Initial query cursor.
+                     */
+                    QueryCursor<K, V> GetInitialQueryCursor()
+                    {
+                        IgniteError err;
+
+                        QueryCursor<K, V> res = GetInitialQueryCursor(err);
+
+                        IgniteError::ThrowIfNeeded(err);
+
+                        return res;
+                    }
+
+                    /**
+                     * Gets the cursor for initial query.
+                     * Can be called only once, results in error on consequent
+                     * calls.
+                     *
+                     * @param err Error.
+                     * @return Initial query cursor.
+                     */
+                    QueryCursor<K, V> GetInitialQueryCursor(IgniteError& err)
+                    {
+                        impl::cache::query::continuous::ContinuousQueryHandleImpl* impl0 = impl.Get();
+
+                        if (impl0)
+                            return QueryCursor<K, V>(impl0->GetInitialQueryCursor(err));
+                        else
+                        {
+                            err = IgniteError(IgniteError::IGNITE_ERR_GENERIC,
+                                "Instance is not usable (did you check for error?).");
+
+                            return QueryCursor<K, V>();
+                        }
+                    }
+
+                    /**
+                     * Check if the instance is valid.
+                     *
+                     * Invalid instance can be returned if some of the previous
+                     * operations have resulted in a failure. For example invalid
+                     * instance can be returned by not-throwing version of method
+                     * in case of error. Invalid instances also often can be
+                     * created using default constructor.
+                     *
+                     * @return True if the instance is valid and can be used.
+                     */
+                    bool IsValid() const
+                    {
+                        return impl.IsValid();
+                    }
+
+                private:
+                    typedef impl::cache::query::continuous::ContinuousQueryHandleImpl ContinuousQueryHandleImpl;
+
+                    /** Implementation delegate. */
+                    common::concurrent::SharedPointer<ContinuousQueryHandleImpl> impl;
+                };
+            }
+        }
+    }
+}
+
+#endif //_IGNITE_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY_HANDLE
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/impl/cache/cache_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/cache/cache_impl.h b/modules/platforms/cpp/core/include/ignite/impl/cache/cache_impl.h
index 3e0f177..535e3ec 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/cache/cache_impl.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/cache/cache_impl.h
@@ -22,7 +22,10 @@
 #include <ignite/cache/query/query_sql.h>
 #include <ignite/cache/query/query_text.h>
 #include <ignite/cache/query/query_sql_fields.h>
+#include <ignite/cache/query/continuous/continuous_query_handle.h>
 #include <ignite/impl/cache/query/query_impl.h>
+#include <ignite/impl/cache/query/continuous/continuous_query_handle_impl.h>
+#include <ignite/impl/cache/query/continuous/continuous_query_impl.h>
 
 #include <ignite/impl/interop/interop_target.h>
 
@@ -309,12 +312,59 @@ namespace ignite
                  * @return Query cursor.
                  */
                 query::QueryCursorImpl* QuerySqlFields(const ignite::cache::query::SqlFieldsQuery& qry, IgniteError* err);
-                
+
+                /**
+                 * Start continuous query execution.
+                 *
+                 * @param qry Continuous query.
+                 * @param err Error.
+                 * @return Continuous query handle.
+                 */
+                query::continuous::ContinuousQueryHandleImpl* QueryContinuous(
+                    const common::concurrent::SharedPointer<query::continuous::ContinuousQueryImplBase> qry,
+                    IgniteError& err);
+
+                /**
+                 * Start continuous query execution with initial query.
+                 *
+                 * @param qry Continuous query.
+                 * @param initialQry Initial query.
+                 * @param err Error.
+                 * @return Continuous query handle.
+                 */
+                query::continuous::ContinuousQueryHandleImpl* QueryContinuous(
+                    const common::concurrent::SharedPointer<query::continuous::ContinuousQueryImplBase> qry,
+                    const ignite::cache::query::SqlQuery& initialQry, IgniteError& err);
+
+                /**
+                 * Start continuous query execution with initial query.
+                 *
+                 * @param qry Continuous query.
+                 * @param initialQry Initial query.
+                 * @param err Error.
+                 * @return Continuous query handle.
+                 */
+                query::continuous::ContinuousQueryHandleImpl* QueryContinuous(
+                    const common::concurrent::SharedPointer<query::continuous::ContinuousQueryImplBase> qry,
+                    const ignite::cache::query::TextQuery& initialQry, IgniteError& err);
+
+                /**
+                 * Start continuous query execution with initial query.
+                 *
+                 * @param qry Continuous query.
+                 * @param initialQry Initial query.
+                 * @param err Error.
+                 * @return Continuous query handle.
+                 */
+                query::continuous::ContinuousQueryHandleImpl* QueryContinuous(
+                    const common::concurrent::SharedPointer<query::continuous::ContinuousQueryImplBase> qry,
+                    const ignite::cache::query::ScanQuery& initialQry, IgniteError& err);
+
             private:
+                IGNITE_NO_COPY_ASSIGNMENT(CacheImpl)
+
                 /** Name. */
                 char* name; 
-                
-                IGNITE_NO_COPY_ASSIGNMENT(CacheImpl)
 
                 /**
                  * Internal query execution routine.
@@ -346,11 +396,67 @@ namespace ignite
                     if (jniErr.code == ignite::java::IGNITE_JNI_ERR_SUCCESS)
                         return new query::QueryCursorImpl(GetEnvironmentPointer(), qryJavaRef);
                     else
-                        return NULL;
+                        return 0;
+                }
+
+                /**
+                 * Start continuous query execution with the initial query.
+                 *
+                 * @param qry Continuous query.
+                 * @param initialQry Initial query to be executed.
+                 * @param err Error.
+                 * @return Continuous query handle.
+                 */
+                template<typename T>
+                query::continuous::ContinuousQueryHandleImpl* QueryContinuous(
+                    const common::concurrent::SharedPointer<query::continuous::ContinuousQueryImplBase> qry,
+                    const T& initialQry, int32_t typ, int32_t cmd, IgniteError& err)
+                {
+                    jni::java::JniErrorInfo jniErr;
+
+                    common::concurrent::SharedPointer<interop::InteropMemory> mem = GetEnvironment().AllocateMemory();
+                    interop::InteropMemory* mem0 = mem.Get();
+                    interop::InteropOutputStream out(mem0);
+                    binary::BinaryWriterImpl writer(&out, GetEnvironment().GetTypeManager());
+                    ignite::binary::BinaryRawWriter rawWriter(&writer);
+
+                    const query::continuous::ContinuousQueryImplBase& qry0 = *qry.Get();
+
+                    int64_t handle = GetEnvironment().GetHandleRegistry().Allocate(qry);
+
+                    rawWriter.WriteInt64(handle);
+                    rawWriter.WriteBool(qry0.GetLocal());
+
+                    // Filters are not supported for now.
+                    rawWriter.WriteBool(false);
+                    rawWriter.WriteNull();
+
+                    rawWriter.WriteInt32(qry0.GetBufferSize());
+                    rawWriter.WriteInt64(qry0.GetTimeInterval());
+
+                    // Autounsubscribe is a filter feature.
+                    rawWriter.WriteBool(false);
+
+                    // Writing initial query. When there is not initial query writing -1.
+                    rawWriter.WriteInt32(typ);
+                    if (typ != -1)
+                        initialQry.Write(rawWriter);
+
+                    out.Synchronize();
+
+                    jobject qryJavaRef = GetEnvironment().Context()->CacheOutOpContinuousQuery(GetTarget(),
+                        cmd, mem.Get()->PointerLong(), &jniErr);
+
+                    IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, &err);
+
+                    if (jniErr.code == java::IGNITE_JNI_ERR_SUCCESS)
+                        return new query::continuous::ContinuousQueryHandleImpl(GetEnvironmentPointer(), handle, qryJavaRef);
+
+                    return 0;
                 }
             };
         }
     }    
 }
 
-#endif
\ No newline at end of file
+#endif

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_handle_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_handle_impl.h b/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_handle_impl.h
new file mode 100644
index 0000000..75504b1
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_handle_impl.h
@@ -0,0 +1,101 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::impl::cache::query::continuous::ContinuousQueryHandleImpl class.
+ */
+
+#ifndef _IGNITE_IMPL_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY_HANDLE_IMPL
+#define _IGNITE_IMPL_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY_HANDLE_IMPL
+
+#include "ignite/cache/query/query_cursor.h"
+#include "ignite/impl/cache/query/continuous/continuous_query_impl.h"
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace cache
+        {
+            namespace query
+            {
+                namespace continuous
+                {
+                    /**
+                     * Continuous query handle implementation.
+                     */
+                    class IGNITE_IMPORT_EXPORT ContinuousQueryHandleImpl
+                    {
+                        typedef common::concurrent::SharedPointer<IgniteEnvironment> SP_IgniteEnvironment;
+                        typedef common::concurrent::SharedPointer<ContinuousQueryImplBase> SP_ContinuousQueryImplBase;
+                    public:
+                        /**
+                         * Default constructor.
+                         * 
+                         * @param env Environment.
+                         * @param javaRef Java reference.
+                         */
+                        ContinuousQueryHandleImpl(SP_IgniteEnvironment env, int64_t handle, jobject javaRef);
+
+                        /**
+                         * Destructor.
+                         */
+                        ~ContinuousQueryHandleImpl();
+
+                        /**
+                         * Gets the cursor for initial query.
+                         * Can be called only once, throws exception on consequent calls.
+                         *
+                         * @param err Error.
+                         * @return Initial query cursor.
+                         */
+                        QueryCursorImpl* GetInitialQueryCursor(IgniteError& err);
+
+                        /**
+                         * Set query to keep pointer to.
+                         *
+                         * @param query Query.
+                         */
+                        void SetQuery(SP_ContinuousQueryImplBase query);
+
+                    private:
+                        /** Environment. */
+                        SP_IgniteEnvironment env;
+
+                        /** Local handle for handle registry. */
+                        int64_t handle;
+
+                        /** Handle to Java object. */
+                        jobject javaRef;
+
+                        /** Shared pointer to query. Kept for query to live long enough. */
+                        SP_ContinuousQueryImplBase qry;
+
+                        /** Mutex. */
+                        common::concurrent::CriticalSection mutex;
+
+                        /** Cursor extracted. */
+                        bool extracted;
+                    };
+                }
+            }
+        }
+    }
+}
+
+#endif //_IGNITE_IMPL_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY_HANDLE_IMPL
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_impl.h b/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_impl.h
new file mode 100644
index 0000000..50ced12
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_impl.h
@@ -0,0 +1,351 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::impl::cache::query::continuous::ContinuousQueryImpl class.
+ */
+
+#ifndef _IGNITE_IMPL_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY_IMPL
+#define _IGNITE_IMPL_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY_IMPL
+
+#include <stdint.h>
+
+#include <ignite/reference.h>
+
+#include <ignite/cache/event/cache_entry_event_listener.h>
+#include <ignite/binary/binary_raw_reader.h>
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace cache
+        {
+            namespace query
+            {
+                namespace continuous
+                {
+                    /**
+                     * Continuous query base implementation class.
+                     *
+                     * Continuous queries allow to register a remote and a listener
+                     * for cache update events. On any update to the related cache
+                     * an event is sent to the node that has executed the query and
+                     * listener is notified on that node.
+                     *
+                     * Continuous query can either be executed on the whole topology
+                     * or only on local node.
+                     *
+                     * To execute the query over the cache use method
+                     * ignite::cache::Cache::QueryContinuous().
+                     */
+                    class ContinuousQueryImplBase
+                    {
+                    public:
+                        /**
+                         * Destructor.
+                         */
+                        virtual ~ContinuousQueryImplBase()
+                        {
+                            // No-op.
+                        }
+
+                        /**
+                         * Default value for the buffer size.
+                         */
+                        enum { DEFAULT_BUFFER_SIZE = 1 };
+
+                        /**
+                         * Default value for the time interval.
+                         */
+                        enum { DEFAULT_TIME_INTERVAL = 0 };
+
+                        /**
+                         * Constructor.
+                         *
+                         * @param loc Whether query should be executed locally.
+                         */
+                        explicit ContinuousQueryImplBase(bool loc) :
+                            local(loc),
+                            bufferSize(DEFAULT_BUFFER_SIZE),
+                            timeInterval(DEFAULT_TIME_INTERVAL)
+                        {
+                            // No-op.
+                        }
+
+                        /**
+                         * Set local flag.
+                         *
+                         * @param val Value of the flag. If true, query will be
+                         *     executed only on local node, so only local entries
+                         *     will be returned as query result.
+                         */
+                        void SetLocal(bool val)
+                        {
+                            local = val;
+                        }
+
+                        /**
+                         * Get local flag.
+                         *
+                         * @return Value of the flag. If true, query will be
+                         *     executed only on local node, so only local entries
+                         *     will be returned as query result.
+                         */
+                        bool GetLocal() const
+                        {
+                            return local;
+                        }
+
+                        /**
+                         * Set buffer size.
+                         *
+                         * When a cache update happens, entry is first
+                         * put into a buffer. Entries from buffer will be sent to
+                         * the master node only if the buffer is full or time
+                         * provided via timeInterval is exceeded.
+                         *
+                         * @param val Buffer size.
+                         */
+                        void SetBufferSize(int32_t val)
+                        {
+                            bufferSize = val;
+                        }
+
+                        /**
+                         * Get buffer size.
+                         *
+                         * When a cache update happens, entry is first
+                         * put into a buffer. Entries from buffer will be sent to
+                         * the master node only if the buffer is full or time
+                         * provided via timeInterval is exceeded.
+                         *
+                         * @return Buffer size.
+                         */
+                        int32_t GetBufferSize() const
+                        {
+                            return bufferSize;
+                        }
+
+                        /**
+                         * Set time interval.
+                         *
+                         * When a cache update happens, entry is first put into
+                         * a buffer. Entries from buffer are sent to the master node
+                         * only if the buffer is full (its size can be changed via
+                         * SetBufferSize) or time provided via this method is
+                         * exceeded.
+                         *
+                         * Default value is DEFAULT_TIME_INTERVAL, i.e. 0, which
+                         * means that time check is disabled and entries will be
+                         * sent only when buffer is full.
+                         *
+                         * @param val Time interval in miliseconds.
+                         */
+                        void SetTimeInterval(int64_t val)
+                        {
+                            timeInterval = val;
+                        }
+
+                        /**
+                         * Get time interval.
+                         *
+                         * When a cache update happens, entry is first put into
+                         * a buffer. Entries from buffer are sent to the master node
+                         * only if the buffer is full (its size can be changed via
+                         * SetBufferSize) or time provided via SetTimeInterval
+                         * method is exceeded.
+                         *
+                         * Default value is DEFAULT_TIME_INTERVAL, i.e. 0, which
+                         * means that time check is disabled and entries will be
+                         * sent only when buffer is full.
+                         *
+                         * @return Time interval.
+                         */
+                        int64_t GetTimeInterval() const
+                        {
+                            return timeInterval;
+                        }
+
+                        /**
+                         * Callback that reads and processes cache events.
+                         *
+                         * @param reader Reader to use.
+                         */
+                        virtual void ReadAndProcessEvents(ignite::binary::BinaryRawReader& reader) = 0;
+
+                    private:
+                        /**
+                         * Local flag. When set query will be executed only on local
+                         * node, so only local entries will be returned as query
+                         * result.
+                         *
+                         * Default value is false.
+                         */
+                        bool local;
+
+                        /**
+                         * Buffer size. When a cache update happens, entry is first
+                         * put into a buffer. Entries from buffer will be sent to
+                         * the master node only if the buffer is full or time
+                         * provided via timeInterval is exceeded.
+                         *
+                         * Default value is DEFAULT_BUFFER_SIZE.
+                         */
+                        int32_t bufferSize;
+
+                        /**
+                         * Time interval in miliseconds. When a cache update
+                         * happens, entry is first put into a buffer. Entries from
+                         * buffer will be sent to the master node only if the buffer
+                         * is full (its size can be changed via SetBufferSize) or
+                         * time provided via SetTimeInterval method is exceeded.
+                         *
+                         * Default value is DEFAULT_TIME_INTERVAL, i.e. 0, which
+                         * means that time check is disabled and entries will be
+                         * sent only when buffer is full.
+                         */
+                        int64_t timeInterval;
+                    };
+
+                    /**
+                     * Continuous query implementation.
+                     *
+                     * Continuous queries allow to register a remote and a listener
+                     * for cache update events. On any update to the related cache
+                     * an event is sent to the node that has executed the query and
+                     * listener is notified on that node.
+                     *
+                     * Continuous query can either be executed on the whole topology
+                     * or only on local node.
+                     *
+                     * To execute the query over the cache use method
+                     * ignite::cache::Cache::QueryContinuous().
+                     */
+                    template<typename K, typename V>
+                    class ContinuousQueryImpl : public ContinuousQueryImplBase
+                    {
+                    public:
+                        /**
+                         * Destructor.
+                         */
+                        virtual ~ContinuousQueryImpl()
+                        {
+                            // No-op.
+                        }
+
+                        /**
+                         * Constructor.
+                         *
+                         * @param lsnr Event listener. Invoked on the node where
+                         *     continuous query execution has been started.
+                         */
+                        ContinuousQueryImpl(Reference<ignite::cache::event::CacheEntryEventListener<K, V>>& lsnr) :
+                            ContinuousQueryImplBase(false),
+                            lsnr(lsnr)
+                        {
+                            // No-op.
+                        }
+
+                        /**
+                         * Constructor.
+                         *
+                         * @param lsnr Event listener Invoked on the node where
+                         *     continuous query execution has been started.
+                         * @param loc Whether query should be executed locally.
+                         */
+                        ContinuousQueryImpl(Reference<ignite::cache::event::CacheEntryEventListener<K, V>>& lsnr, bool loc) :
+                            ContinuousQueryImplBase(loc),
+                            lsnr(lsnr)
+                        {
+                            // No-op.
+                        }
+
+                        /**
+                         * Set cache entry event listener.
+                         *
+                         * @param val Cache entry event listener. Invoked on the
+                         *     node where continuous query execution has been
+                         *     started.
+                         */
+                        void SetListener(Reference<ignite::cache::event::CacheEntryEventListener<K, V>>& val)
+                        {
+                            lsnr = val;
+                        }
+
+                        /**
+                         * Check if the query has listener.
+                         *
+                         * @return True if the query has listener.
+                         */
+                        bool HasListener() const
+                        {
+                            return !lsnr.IsNull();
+                        }
+
+                        /**
+                         * Get cache entry event listener.
+                         *
+                         * @return Cache entry event listener.
+                         */
+                        const ignite::cache::event::CacheEntryEventListener<K, V>& GetListener() const
+                        {
+                            return lsnr.Get();
+                        }
+
+                        /**
+                         * Get cache entry event listener.
+                         *
+                         * @return Cache entry event listener.
+                         */
+                        ignite::cache::event::CacheEntryEventListener<K, V>& GetListener()
+                        {
+                            return lsnr.Get();
+                        }
+
+                        /**
+                         * Callback that reads and processes cache events.
+                         *
+                         * @param reader Reader to use.
+                         */
+                        virtual void ReadAndProcessEvents(ignite::binary::BinaryRawReader& reader)
+                        {
+                            // Number of events.
+                            int32_t cnt = reader.ReadInt32();
+
+                            // Storing events here.
+                            std::vector< ignite::cache::CacheEntryEvent<K, V> > events;
+                            events.resize(cnt);
+
+                            for (int32_t i = 0; i < cnt; ++i)
+                                events[i].Read(reader);
+
+                            lsnr.Get().OnEvent(events.data(), cnt);
+                        }
+
+                    private:
+                        /** Cache entry event listener. */
+                        Reference<ignite::cache::event::CacheEntryEventListener<K, V>> lsnr;
+                    };
+                }
+            }
+        }
+    }
+}
+
+#endif //_IGNITE_IMPL_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY_IMPL
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/impl/handle_registry.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/handle_registry.h b/modules/platforms/cpp/core/include/ignite/impl/handle_registry.h
index 107042a..3c4d123 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/handle_registry.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/handle_registry.h
@@ -28,21 +28,9 @@ namespace ignite
     namespace impl
     {
         /**
-         * Something what can be registered inside handle registry.
-         */
-        class IGNITE_IMPORT_EXPORT HandleRegistryEntry
-        {
-        public:
-            /**
-             * Destructor.
-             */
-            virtual ~HandleRegistryEntry();
-        };
-
-        /**
          * Handle registry segment containing thread-specific data for slow-path access.
          */
-        class IGNITE_IMPORT_EXPORT HandleRegistrySegment
+        class HandleRegistrySegment
         {
         public:
             /**
@@ -61,7 +49,7 @@ namespace ignite
              * @param hnd Handle.
              * @return Associated entry or NULL if it doesn't exists.
              */
-            ignite::common::concurrent::SharedPointer<HandleRegistryEntry> Get(int64_t hnd);
+            common::concurrent::SharedPointer<void> Get(int64_t hnd);
 
             /**
              * Put entry into segment.
@@ -69,14 +57,14 @@ namespace ignite
              * @param hnd Handle.
              * @param entry Associated entry (cannot be NULL).
              */
-            void Put(int64_t hnd, const ignite::common::concurrent::SharedPointer<HandleRegistryEntry>& entry);
+            void Put(int64_t hnd, const common::concurrent::SharedPointer<void>& entry);
 
             /**
              * Remove entry from the segment.
              *
              * @param hnd Handle.
              */
-            void Remove(int64_t hnd);            
+            void Remove(int64_t hnd);
 
             /**
              * Clear all entries from the segment.
@@ -84,10 +72,10 @@ namespace ignite
             void Clear();
         private:
             /** Map with data. */
-            std::map<int64_t, ignite::common::concurrent::SharedPointer<HandleRegistryEntry>>* map;
+            std::map<int64_t, common::concurrent::SharedPointer<void>> map;
 
             /** Mutex. */
-            ignite::common::concurrent::CriticalSection* mux;
+            common::concurrent::CriticalSection mux;
 
             IGNITE_NO_COPY_ASSIGNMENT(HandleRegistrySegment);
         };
@@ -102,7 +90,7 @@ namespace ignite
              * Constructor.
              *
              * @param fastCap Fast-path capacity.
-             * @param segmentCnt Slow-path segments count.
+             * @param slowSegmentCnt Slow-path segments count.
              */
             HandleRegistry(int32_t fastCap, int32_t slowSegmentCnt);
 
@@ -117,7 +105,7 @@ namespace ignite
              * @param target Target.
              * @return Handle.
              */
-            int64_t Allocate(const ignite::common::concurrent::SharedPointer<HandleRegistryEntry>& target);
+            int64_t Allocate(const common::concurrent::SharedPointer<void>& target);
 
             /**
              * Allocate handle in critical mode.
@@ -125,7 +113,7 @@ namespace ignite
              * @param target Target.
              * @return Handle.
              */
-            int64_t AllocateCritical(const ignite::common::concurrent::SharedPointer<HandleRegistryEntry>& target);
+            int64_t AllocateCritical(const common::concurrent::SharedPointer<void>& target);
 
             /**
              * Allocate handle in safe mode.
@@ -133,7 +121,7 @@ namespace ignite
              * @param target Target.
              * @return Handle.
              */
-            int64_t AllocateSafe(const ignite::common::concurrent::SharedPointer<HandleRegistryEntry>& target);
+            int64_t AllocateSafe(const common::concurrent::SharedPointer<void>& target);
 
             /**
              * Allocate handle in critical and safe modes.
@@ -141,7 +129,7 @@ namespace ignite
              * @param target Target.
              * @return Handle.
              */
-            int64_t AllocateCriticalSafe(const ignite::common::concurrent::SharedPointer<HandleRegistryEntry>& target);
+            int64_t AllocateCriticalSafe(const common::concurrent::SharedPointer<void>& target);
 
             /**
              * Release handle.
@@ -154,35 +142,36 @@ namespace ignite
              * Get target.
              *
              * @param hnd Handle.
-             * @param Target.
+             * @return Target.
              */
-            ignite::common::concurrent::SharedPointer<HandleRegistryEntry> Get(int64_t hnd);
+            common::concurrent::SharedPointer<void> Get(int64_t hnd);
 
             /**
              * Close the registry.
              */
             void Close();
+
         private:
             /** Fast-path container capacity. */
-            int32_t fastCap;                     
+            int32_t fastCap;
 
             /** Fast-path counter. */
-            int32_t fastCtr;               
+            int32_t fastCtr;
 
             /** Fast-path container. */
-            ignite::common::concurrent::SharedPointer<HandleRegistryEntry>* fast;
+            common::concurrent::SharedPointer<void>* fast;
 
             /** Amount of slow-path segments. */
-            int32_t slowSegmentCnt;            
+            int32_t slowSegmentCnt;
 
             /** Slow-path counter. */
-            int64_t slowCtr;                                                         
-            
+            int64_t slowCtr;
+
             /** Slow-path segments. */
-            HandleRegistrySegment** slow;                                            
+            HandleRegistrySegment** slow;
 
             /** Close flag. */
-            int32_t closed;                                                           
+            int32_t closed;
 
             IGNITE_NO_COPY_ASSIGNMENT(HandleRegistry);
 
@@ -190,11 +179,10 @@ namespace ignite
              * Internal allocation routine.
              *
              * @param target Target.
-             * @param Critical mode flag.
-             * @param Safe mode flag.
+             * @param critical mode flag.
+             * @param safe mode flag.
              */
-            int64_t Allocate0(const ignite::common::concurrent::SharedPointer<HandleRegistryEntry>& target,
-                bool critical, bool safe);
+            int64_t Allocate0(const common::concurrent::SharedPointer<void>& target, bool critical, bool safe);
         };
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h b/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
index fb6f657..2b2a117 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
@@ -20,14 +20,15 @@
 
 #include <ignite/common/concurrent.h>
 #include <ignite/jni/java.h>
+#include <ignite/jni/utils.h>
 
 #include "ignite/impl/interop/interop_memory.h"
 #include "ignite/impl/binary/binary_type_manager.h"
-#include "ignite/jni/utils.h"
+#include "ignite/impl/handle_registry.h"
 
-namespace ignite 
+namespace ignite
 {
-    namespace impl 
+    namespace impl
     {
         /**
          * Defines environment in which Ignite operates.
@@ -41,6 +42,16 @@ namespace ignite
             enum { DEFAULT_ALLOCATION_SIZE = 1024 };
 
             /**
+             * Default fast path handle registry containers capasity.
+             */
+            enum { DEFAULT_FAST_PATH_CONTAINERS_CAP = 1024 };
+
+            /**
+            * Default slow path handle registry containers capasity.
+            */
+            enum { DEFAULT_SLOW_PATH_CONTAINERS_CAP = 16 };
+
+            /**
              * Default constructor.
              */
             IgniteEnvironment();
@@ -78,6 +89,13 @@ namespace ignite
             void OnStartCallback(long long memPtr, jobject proc);
 
             /**
+             * Continuous query listener apply callback.
+             *
+             * @param mem Memory with data.
+             */
+            void OnContinuousQueryListenerApply(common::concurrent::SharedPointer<interop::InteropMemory>& mem);
+
+            /**
              * Get name of Ignite instance.
              *
              * @return Name.
@@ -133,6 +151,13 @@ namespace ignite
              */
             void ProcessorReleaseStart();
 
+            /**
+             * Get handle registry.
+             *
+             * @return Handle registry.
+             */
+            HandleRegistry& GetHandleRegistry();
+
         private:
             /** Context to access Java. */
             common::concurrent::SharedPointer<jni::java::JniContext> ctx;
@@ -152,6 +177,9 @@ namespace ignite
             /** Type updater. */
             binary::BinaryTypeUpdater* metaUpdater;
 
+            /** Handle registry. */
+            HandleRegistry registry;
+
             IGNITE_NO_COPY_ASSIGNMENT(IgniteEnvironment);
         };
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/namespaces.dox
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/namespaces.dox b/modules/platforms/cpp/core/namespaces.dox
index 0f5f11f..49379e6 100644
--- a/modules/platforms/cpp/core/namespaces.dox
+++ b/modules/platforms/cpp/core/namespaces.dox
@@ -22,40 +22,54 @@
  * computing and transacting on large-scale data sets in real-time, orders of magnitude faster than possible with
  * traditional disk-based or flash-based technologies.
  */
- 
+
  /**
   * Apache %Ignite API.
   */
  namespace ignite
  {
-	 /**
-	  * %Ignite Binary Objects API.
-	  */
-	 namespace binary
-	 {
-		 // Empty.
-	 }
+     /**
+      * %Ignite Binary Objects API.
+      */
+     namespace binary
+     {
+         // Empty.
+     }
+
+     /**
+      * %Ignite %Transaction API.
+      */
+     namespace transactions
+     {
+         // Empty.
+     }
 
-	 /**
-	  * %Ignite %Transaction API.
-	  */
-	 namespace transactions
-	 {
-		 // Empty.
-	 }
-	 
-	 /**
-	  * %Ignite %Cache API.
-	  */
-	 namespace cache
-	 {
-		 /**
-		  * Contains APIs for creating and executing cache queries.
-		  */
-		 namespace query
-		 {
-			 // Empty.
-		 }
-	 }
+     /**
+      * %Ignite %Cache API.
+      */
+     namespace cache
+     {
+        /**
+         * Contains APIs for cache events.
+         */
+        namespace event
+        {
+            // Empty.
+        }
+
+         /**
+          * Contains APIs for creating and executing cache queries.
+          */
+         namespace query
+         {
+            /**
+             * Contains APIs for continuous queries.
+             */
+            namespace continuous
+            {
+                // Empty.
+            }
+         }
+     }
  }
- 
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/project/vs/core.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/project/vs/core.vcxproj b/modules/platforms/cpp/core/project/vs/core.vcxproj
index 6320323..89a2dff 100644
--- a/modules/platforms/cpp/core/project/vs/core.vcxproj
+++ b/modules/platforms/cpp/core/project/vs/core.vcxproj
@@ -193,6 +193,10 @@
     <ClInclude Include="..\..\include\ignite\cache\cache.h" />
     <ClInclude Include="..\..\include\ignite\cache\cache_entry.h" />
     <ClInclude Include="..\..\include\ignite\cache\cache_peek_mode.h" />
+    <ClInclude Include="..\..\include\ignite\cache\event\cache_entry_event.h" />
+    <ClInclude Include="..\..\include\ignite\cache\event\cache_entry_event_listener.h" />
+    <ClInclude Include="..\..\include\ignite\cache\query\continuous\continuous_query.h" />
+    <ClInclude Include="..\..\include\ignite\cache\query\continuous\continuous_query_handle.h" />
     <ClInclude Include="..\..\include\ignite\cache\query\query.h" />
     <ClInclude Include="..\..\include\ignite\cache\query\query_argument.h" />
     <ClInclude Include="..\..\include\ignite\cache\query\query_cursor.h" />
@@ -208,6 +212,8 @@
     <ClInclude Include="..\..\include\ignite\impl\binary\binary_type_updater_impl.h" />
     <ClInclude Include="..\..\include\ignite\impl\cache\cache_impl.h" />
     <ClInclude Include="..\..\include\ignite\impl\cache\query\query_batch.h" />
+    <ClInclude Include="..\..\include\ignite\impl\cache\query\continuous\continuous_query_handle_impl.h" />
+    <ClInclude Include="..\..\include\ignite\impl\cache\query\continuous\continuous_query_impl.h" />
     <ClInclude Include="..\..\include\ignite\impl\cache\query\query_fields_row_impl.h" />
     <ClInclude Include="..\..\include\ignite\impl\cache\query\query_impl.h" />
     <ClInclude Include="..\..\include\ignite\impl\ignite_environment.h" />
@@ -229,6 +235,7 @@
     <ClCompile Include="..\..\src\impl\binary\binary_type_updater_impl.cpp" />
     <ClCompile Include="..\..\src\impl\cache\cache_impl.cpp" />
     <ClCompile Include="..\..\src\impl\cache\query\query_batch.cpp" />
+    <ClCompile Include="..\..\src\impl\cache\query\continuous\continuous_query_handle_impl.cpp" />
     <ClCompile Include="..\..\src\impl\cache\query\query_impl.cpp" />
     <ClCompile Include="..\..\src\impl\ignite_environment.cpp" />
     <ClCompile Include="..\..\src\impl\ignite_impl.cpp" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/project/vs/core.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/project/vs/core.vcxproj.filters b/modules/platforms/cpp/core/project/vs/core.vcxproj.filters
index c5fb532..9cb5f78 100644
--- a/modules/platforms/cpp/core/project/vs/core.vcxproj.filters
+++ b/modules/platforms/cpp/core/project/vs/core.vcxproj.filters
@@ -46,6 +46,9 @@
     <ClCompile Include="..\..\src\impl\cache\query\query_batch.cpp">
       <Filter>Code\impl\cache\query</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\impl\cache\query\continuous\continuous_query_handle_impl.cpp">
+      <Filter>Code\impl\cache\query\continuous</Filter>
+    </ClCompile>
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\..\include\ignite\impl\cache\cache_impl.h">
@@ -144,6 +147,24 @@
     <ClInclude Include="..\..\include\ignite\impl\cache\query\query_batch.h">
       <Filter>Code\impl\cache\query</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\include\ignite\cache\query\continuous\continuous_query_handle.h">
+      <Filter>Code\cache\query\continuous</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\cache\query\continuous\continuous_query.h">
+      <Filter>Code\cache\query\continuous</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\impl\cache\query\continuous\continuous_query_handle_impl.h">
+      <Filter>Code\impl\cache\query\continuous</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\cache\event\cache_entry_event_listener.h">
+      <Filter>Code\cache\event</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\cache\event\cache_entry_event.h">
+      <Filter>Code\cache\event</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\impl\cache\query\continuous\continuous_query_impl.h">
+      <Filter>Code\impl\cache\query\continuous</Filter>
+    </ClInclude>
   </ItemGroup>
   <ItemGroup>
     <Filter Include="Code">
@@ -176,5 +197,14 @@
     <Filter Include="Code\transactions">
       <UniqueIdentifier>{146fe661-0ad3-4d51-83a3-fce8a897e84d}</UniqueIdentifier>
     </Filter>
+    <Filter Include="Code\cache\query\continuous">
+      <UniqueIdentifier>{2056dfc8-4ced-4658-b2b7-a8c81a7ef797}</UniqueIdentifier>
+    </Filter>
+    <Filter Include="Code\impl\cache\query\continuous">
+      <UniqueIdentifier>{d633f819-7b30-4e26-81ec-f708d1c1ff8e}</UniqueIdentifier>
+    </Filter>
+    <Filter Include="Code\cache\event">
+      <UniqueIdentifier>{e03c3690-ff22-4c78-83a0-b77cebb7f980}</UniqueIdentifier>
+    </Filter>
   </ItemGroup>
 </Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp b/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp
index 0630921..ad69d45 100644
--- a/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp
+++ b/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp
@@ -26,9 +26,11 @@ using namespace ignite::java;
 using namespace ignite::common;
 using namespace ignite::cache;
 using namespace ignite::cache::query;
+using namespace ignite::cache::query::continuous;
 using namespace ignite::impl;
 using namespace ignite::impl::binary;
 using namespace ignite::impl::cache::query;
+using namespace ignite::impl::cache::query::continuous;
 using namespace ignite::impl::interop;
 using namespace ignite::binary;
 
@@ -89,6 +91,9 @@ namespace ignite
             /** Operation: PutIfAbsent. */
             const int32_t OP_PUT_IF_ABSENT = 28;
 
+            /** Operation: CONTINUOUS query. */
+            const int32_t OP_QRY_CONTINUOUS = 29;
+
             /** Operation: SCAN query. */
             const int32_t OP_QRY_SCAN = 30;
 
@@ -301,6 +306,32 @@ namespace ignite
             {
                 return QueryInternal(qry, OP_QRY_SQL_FIELDS, err);
             }
+
+            ContinuousQueryHandleImpl* CacheImpl::QueryContinuous(const SharedPointer<ContinuousQueryImplBase> qry,
+                const SqlQuery& initialQry, IgniteError& err)
+            {
+                return QueryContinuous(qry, initialQry, OP_QRY_SQL, OP_QRY_CONTINUOUS, err);
+            }
+
+            ContinuousQueryHandleImpl* CacheImpl::QueryContinuous(const SharedPointer<ContinuousQueryImplBase> qry,
+                const TextQuery& initialQry, IgniteError& err)
+            {
+                return QueryContinuous(qry, initialQry, OP_QRY_TEXT, OP_QRY_CONTINUOUS, err);
+            }
+
+            ContinuousQueryHandleImpl* CacheImpl::QueryContinuous(const SharedPointer<ContinuousQueryImplBase> qry,
+                const ScanQuery& initialQry, IgniteError& err)
+            {
+                return QueryContinuous(qry, initialQry, OP_QRY_SCAN, OP_QRY_CONTINUOUS, err);
+            }
+
+            ContinuousQueryHandleImpl* CacheImpl::QueryContinuous(const SharedPointer<ContinuousQueryImplBase> qry,
+                IgniteError& err)
+            {
+                struct { void Write(BinaryRawWriter&) const { }} dummy;
+
+                return QueryContinuous(qry, dummy, -1, OP_QRY_CONTINUOUS, err);
+            }
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/src/impl/cache/query/continuous/continuous_query_handle_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/cache/query/continuous/continuous_query_handle_impl.cpp b/modules/platforms/cpp/core/src/impl/cache/query/continuous/continuous_query_handle_impl.cpp
new file mode 100644
index 0000000..04e64c9
--- /dev/null
+++ b/modules/platforms/cpp/core/src/impl/cache/query/continuous/continuous_query_handle_impl.cpp
@@ -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.
+ */
+
+#include "ignite/impl/cache/query/continuous/continuous_query_handle_impl.h"
+
+using namespace ignite::common::concurrent;
+using namespace ignite::jni::java;
+using namespace ignite::java;
+using namespace ignite::impl::interop;
+using namespace ignite::impl::binary;
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace cache
+        {
+            namespace query
+            {
+                namespace continuous
+                {
+                    enum Command
+                    {
+                        GET_INITIAL_QUERY = 0,
+
+                        CLOSE = 1
+                    };
+
+                    ContinuousQueryHandleImpl::ContinuousQueryHandleImpl(SP_IgniteEnvironment env, int64_t handle, jobject javaRef) :
+                        env(env),
+                        handle(handle),
+                        javaRef(javaRef),
+                        mutex(),
+                        extracted(false)
+                    {
+                        // No-op.
+                    }
+
+                    ContinuousQueryHandleImpl::~ContinuousQueryHandleImpl()
+                    {
+                        env.Get()->Context()->TargetInLongOutLong(javaRef, CLOSE, 0);
+
+                        JniContext::Release(javaRef);
+
+                        env.Get()->GetHandleRegistry().Release(handle);
+                    }
+
+                    QueryCursorImpl* ContinuousQueryHandleImpl::GetInitialQueryCursor(IgniteError& err)
+                    {
+                        CsLockGuard guard(mutex);
+
+                        if (extracted)
+                        {
+                            err = IgniteError(IgniteError::IGNITE_ERR_GENERIC,
+                                "GetInitialQueryCursor() can be called only once.");
+
+                            return 0;
+                        }
+
+                        JniErrorInfo jniErr;
+
+                        jobject res = env.Get()->Context()->TargetOutObject(javaRef, GET_INITIAL_QUERY, &jniErr);
+
+                        IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, &err);
+
+                        if (jniErr.code != IGNITE_JNI_ERR_SUCCESS)
+                            return 0;
+
+                        extracted = true;
+
+                        return new QueryCursorImpl(env, res);
+                    }
+
+                    void ContinuousQueryHandleImpl::SetQuery(SP_ContinuousQueryImplBase query)
+                    {
+                        qry = query;
+                    }
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/src/impl/handle_registry.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/handle_registry.cpp b/modules/platforms/cpp/core/src/impl/handle_registry.cpp
index c447faa..069e996 100644
--- a/modules/platforms/cpp/core/src/impl/handle_registry.cpp
+++ b/modules/platforms/cpp/core/src/impl/handle_registry.cpp
@@ -23,83 +23,67 @@ namespace ignite
 {
     namespace impl
     {
-        HandleRegistryEntry::~HandleRegistryEntry()
-        {
-            // No-op.
-        }
-
         HandleRegistrySegment::HandleRegistrySegment() : 
-            map(new std::map<int64_t, SharedPointer<HandleRegistryEntry>>()), mux(new CriticalSection())
+            map(),
+            mux()
         {
             // No-op.
         }
 
         HandleRegistrySegment::~HandleRegistrySegment()
         {
-            delete map;
-            delete mux;
+            // No-op.
         }
 
-        SharedPointer<HandleRegistryEntry> HandleRegistrySegment::Get(int64_t hnd)
+        SharedPointer<void> HandleRegistrySegment::Get(int64_t hnd)
         {
-            mux->Enter();
+            typedef std::map<int64_t, SharedPointer<void>> Map;
 
-            SharedPointer<HandleRegistryEntry> res = (*map)[hnd];
+            CsLockGuard guard(mux);
 
-            mux->Leave();
+            Map::const_iterator it = map.find(hnd);
+            if (it == map.end())
+                return SharedPointer<void>();
 
-            return res;
+            return it->second;
         }
 
-        void HandleRegistrySegment::Put(int64_t hnd, const SharedPointer<HandleRegistryEntry>& entry)
+        void HandleRegistrySegment::Put(int64_t hnd, const SharedPointer<void>& entry)
         {
-            mux->Enter();
-
-            (*map)[hnd] = entry;
+            CsLockGuard guard(mux);
 
-            mux->Leave();
+            map[hnd] = entry;
         }
 
         void HandleRegistrySegment::Remove(int64_t hnd)
         {
-            mux->Enter();
+            CsLockGuard guard(mux);
 
-            map->erase(hnd);
-
-            mux->Leave();
+            map.erase(hnd);
         }
 
         void HandleRegistrySegment::Clear()
         {
-            mux->Enter();
-
-            map->erase(map->begin(), map->end());
+            CsLockGuard guard(mux);
 
-            mux->Leave();
+            map.clear();
         }
 
-        HandleRegistry::HandleRegistry(int32_t fastCap, int32_t slowSegmentCnt)
+        HandleRegistry::HandleRegistry(int32_t fastCap, int32_t slowSegmentCnt) :
+            fastCap(fastCap),
+            fastCtr(0),
+            fast(new SharedPointer<void>[fastCap]),
+            slowSegmentCnt(slowSegmentCnt),
+            slowCtr(fastCap),
+            slow(new HandleRegistrySegment*[slowSegmentCnt]),
+            closed(0)
         {
-            this->fastCap = fastCap;
+            for (int32_t i = 0; i < fastCap; i++)
+                fast[i] = SharedPointer<void>();
 
-            fastCtr = 0;
-
-            fast = new SharedPointer<HandleRegistryEntry>[fastCap];
-
-            for (int i = 0; i < fastCap; i++)
-                fast[i] = SharedPointer<HandleRegistryEntry>();
-
-            this->slowSegmentCnt = slowSegmentCnt;
-
-            slowCtr = fastCap;
-
-            slow = new HandleRegistrySegment*[slowSegmentCnt];
-
-            for (int i = 0; i < slowSegmentCnt; i++)
+            for (int32_t i = 0; i < slowSegmentCnt; i++)
                 slow[i] = new HandleRegistrySegment();
 
-            closed = 0;
-
             Memory::Fence();
         }
 
@@ -115,22 +99,22 @@ namespace ignite
             delete[] slow;
         }
 
-        int64_t HandleRegistry::Allocate(const SharedPointer<HandleRegistryEntry>& target)
+        int64_t HandleRegistry::Allocate(const SharedPointer<void>& target)
         {
             return Allocate0(target, false, false);
         }
 
-        int64_t HandleRegistry::AllocateCritical(const SharedPointer<HandleRegistryEntry>& target)
+        int64_t HandleRegistry::AllocateCritical(const SharedPointer<void>& target)
         {
             return Allocate0(target, true, false);
         }
 
-        int64_t HandleRegistry::AllocateSafe(const SharedPointer<HandleRegistryEntry>& target)
+        int64_t HandleRegistry::AllocateSafe(const SharedPointer<void>& target)
         {
             return Allocate0(target, false, true);
         }
 
-        int64_t HandleRegistry::AllocateCriticalSafe(const SharedPointer<HandleRegistryEntry>& target)
+        int64_t HandleRegistry::AllocateCriticalSafe(const SharedPointer<void>& target)
         {
             return Allocate0(target, true, true);
         }
@@ -138,10 +122,10 @@ namespace ignite
         void HandleRegistry::Release(int64_t hnd)
         {
             if (hnd < fastCap)
-                fast[static_cast<int32_t>(hnd)] = SharedPointer<HandleRegistryEntry>();
+                fast[static_cast<int32_t>(hnd)] = SharedPointer<void>();
             else
             {
-                HandleRegistrySegment* segment = *(slow + hnd % slowSegmentCnt);
+                HandleRegistrySegment* segment = slow[hnd % slowSegmentCnt];
 
                 segment->Remove(hnd);
             }
@@ -149,7 +133,7 @@ namespace ignite
             Memory::Fence();
         }
 
-        SharedPointer<HandleRegistryEntry> HandleRegistry::Get(int64_t hnd)
+        SharedPointer<void> HandleRegistry::Get(int64_t hnd)
         {
             Memory::Fence();
 
@@ -157,7 +141,7 @@ namespace ignite
                 return fast[static_cast<int32_t>(hnd)];
             else
             {
-                HandleRegistrySegment* segment = *(slow + hnd % slowSegmentCnt);
+                HandleRegistrySegment* segment = slow[hnd % slowSegmentCnt];
 
                 return segment->Get(hnd);
             }
@@ -168,16 +152,16 @@ namespace ignite
             if (Atomics::CompareAndSet32(&closed, 0, 1))
             {
                 // Cleanup fast-path handles.
-                for (int i = 0; i < fastCap; i++)
-                    fast[i] = SharedPointer<HandleRegistryEntry>();
+                for (int32_t i = 0; i < fastCap; i++)
+                    fast[i] = SharedPointer<void>();
 
                 // Cleanup slow-path handles.
-                for (int i = 0; i < slowSegmentCnt; i++)
-                    (*(slow + i))->Clear();
+                for (int32_t i = 0; i < slowSegmentCnt; i++)
+                    slow[i]->Clear();
             }
         }
 
-        int64_t HandleRegistry::Allocate0(const SharedPointer<HandleRegistryEntry>& target, bool critical, bool safe)
+        int64_t HandleRegistry::Allocate0(const SharedPointer<void>& target, bool critical, bool safe)
         {
             // Check closed state.
             Memory::Fence();
@@ -201,7 +185,7 @@ namespace ignite
 
                         if (safe && closed == 1)
                         {
-                            fast[fastIdx] = SharedPointer<HandleRegistryEntry>();
+                            fast[fastIdx] = SharedPointer<void>();
 
                             return -1;
                         }
@@ -214,7 +198,7 @@ namespace ignite
             // Either allocating on slow-path, or fast-path can no longer accomodate more entries.
             int64_t slowIdx = Atomics::IncrementAndGet64(&slowCtr) - 1;
 
-            HandleRegistrySegment* segment = *(slow + slowIdx % slowSegmentCnt);
+            HandleRegistrySegment* segment = slow[slowIdx % slowSegmentCnt];
 
             segment->Put(slowIdx, target);
 


[15/16] ignite git commit: IGNITE-4469: Hadoop: set default value of offheap memory page and shuffle messages to 1Mb.

Posted by vo...@apache.org.
IGNITE-4469: Hadoop: set default value of offheap memory page and shuffle messages to 1Mb.


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

Branch: refs/heads/ignite-2.0
Commit: 854d1a5d9ccf4d24bd3cdcbb1a60a52c78ef5c88
Parents: 598b464
Author: devozerov <vo...@gridgain.com>
Authored: Wed Dec 21 13:08:30 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Dec 21 13:08:30 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/hadoop/HadoopJobProperty.java    | 4 ++--
 .../internal/processors/hadoop/shuffle/HadoopShuffleJob.java    | 2 +-
 .../hadoop/shuffle/collections/HadoopMultimapBase.java          | 5 ++++-
 3 files changed, 7 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/854d1a5d/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
index 9dd430b..4398acd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
@@ -71,7 +71,7 @@ public enum HadoopJobProperty {
     /**
      * Size in bytes of single memory page which will be allocated for data structures in shuffle.
      * <p>
-     * By default is {@code 32 * 1024}.
+     * Defaults to {@code 1Mb}.
      */
     SHUFFLE_OFFHEAP_PAGE_SIZE("ignite.shuffle.offheap.page.size"),
 
@@ -98,7 +98,7 @@ public enum HadoopJobProperty {
     /**
      * Defines approximate size in bytes of shuffle message which will be passed over wire from mapper to reducer.
      * <p>
-     * Defaults to 128Kb.
+     * Defaults to {@code 1Mb}.
      */
     SHUFFLE_MSG_SIZE("ignite.shuffle.message.size"),
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/854d1a5d/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
index 214a335..1c546a1 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
@@ -76,7 +76,7 @@ import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.get
  */
 public class HadoopShuffleJob<T> implements AutoCloseable {
     /** */
-    private static final int DFLT_SHUFFLE_MSG_SIZE = 128 * 1024;
+    private static final int DFLT_SHUFFLE_MSG_SIZE = 1024 * 1024;
 
     /** */
     private final HadoopJob job;

http://git-wip-us.apache.org/repos/asf/ignite/blob/854d1a5d/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java
index 39b7c51..d7a4541 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java
@@ -40,6 +40,9 @@ import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.get
  * Base class for all multimaps.
  */
 public abstract class HadoopMultimapBase implements HadoopMultimap {
+    /** Default offheap page size. */
+    private static final int DFLT_OFFHEAP_PAGE_SIZE = 1024 * 1024;
+
     /** */
     protected final GridUnsafeMemory mem;
 
@@ -59,7 +62,7 @@ public abstract class HadoopMultimapBase implements HadoopMultimap {
 
         this.mem = mem;
 
-        pageSize = get(jobInfo, SHUFFLE_OFFHEAP_PAGE_SIZE, 32 * 1024);
+        pageSize = get(jobInfo, SHUFFLE_OFFHEAP_PAGE_SIZE, DFLT_OFFHEAP_PAGE_SIZE);
     }
 
     /**


[05/16] ignite git commit: IGNITE-4446: Split query tests into short and long; moved H2 tests into query suite; optimized execution time for one long-running test.

Posted by vo...@apache.org.
IGNITE-4446: Split query tests into short and long; moved H2 tests into query suite; optimized execution time for one long-running test.


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

Branch: refs/heads/ignite-2.0
Commit: 1e97d78c9e5423656ed690b6fefd3b8c96330fb9
Parents: a38ae7e
Author: devozerov <vo...@gridgain.com>
Authored: Mon Dec 19 11:06:24 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Dec 19 11:06:24 2016 +0300

----------------------------------------------------------------------
 .../cache/IgniteCacheLargeResultSelfTest.java   |  15 ++-
 .../IgniteCacheQuerySelfTestSuite.java          | 130 +++++++++++++++----
 .../IgniteCacheQuerySelfTestSuite2.java         | 111 ++++------------
 .../IgniteH2IndexingSpiTestSuite.java           |  16 +--
 4 files changed, 140 insertions(+), 132 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1e97d78c/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLargeResultSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLargeResultSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLargeResultSelfTest.java
index 7a1592c..af2425b 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLargeResultSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLargeResultSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import java.util.List;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
@@ -78,12 +79,18 @@ public class IgniteCacheLargeResultSelfTest extends GridCommonAbstractTest {
      */
     public void testLargeResult() {
         // Fill cache.
-        IgniteCache<Integer,Integer> c = ignite(0).cache(null);
+        try (IgniteDataStreamer<Integer, Integer> streamer = ignite(0).dataStreamer(null)) {
+            streamer.perNodeBufferSize(20000);
 
-        for (int i = 0; i < 50_000; i++)  // default max merge table size is 10000
-            c.put(i, i);
+            for (int i = 0; i < 50_000; i++)  // default max merge table size is 10000
+                streamer.addData(i, i);
 
-        try(QueryCursor<List<?>> res = c.query(
+            streamer.flush();
+        }
+
+        IgniteCache<Integer, Integer> cache = ignite(0).cache(null);
+
+        try(QueryCursor<List<?>> res = cache.query(
             new SqlFieldsQuery("select _val from Integer where _key between ? and ?")
                 .setArgs(10_000, 40_000))){
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1e97d78c/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 783f831..1e98eb5 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
@@ -18,8 +18,23 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
+import org.apache.ignite.internal.processors.cache.CacheLocalQueryDetailMetricsSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheLocalQueryMetricsSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheOffheapBatchIndexingSingleTypeTest;
+import org.apache.ignite.internal.processors.cache.CachePartitionedQueryDetailMetricsDistributedSelfTest;
+import org.apache.ignite.internal.processors.cache.CachePartitionedQueryDetailMetricsLocalSelfTest;
+import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsDistributedSelfTest;
+import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsLocalSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheQueryNewClientSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheQueryOffheapEvictDataLostTest;
+import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryDetailMetricsDistributedSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryDetailMetricsLocalSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsDistributedSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsLocalSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheSqlQueryValueCopySelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheCrossCacheQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexDisabledSelfTest;
+import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexingDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheQueryInternalKeysSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheQuerySerializationSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectFieldsQuerySelfTest;
@@ -29,57 +44,66 @@ import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectQueryArgume
 import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectQueryArgumentsTest;
 import org.apache.ignite.internal.processors.cache.IgniteBinaryWrappedObjectFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheCollocatedQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheCrossCacheJoinRandomTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDeleteSqlQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinCollocatedAndNotTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinCustomAffinityMapper;
+import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinNoIndexTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinPartitionedAndReplicatedTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinQueryConditionsTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDuplicateEntityConfigurationSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinNoIndexTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheFieldsQueryNoDataSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheInsertSqlQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheJoinPartitionedAndReplicatedTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheJoinQueryWithAffinityKeyTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheLargeResultSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheMergeSqlQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheNoClassQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapEvictQueryTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapIndexScanTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapTieredMultithreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCachePartitionedQueryMultiThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingQueryErrorTest;
 import org.apache.ignite.internal.processors.cache.IgniteCachePrimitiveFieldsQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheQueryEvictsMultiThreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheQueryH2IndexingLeakTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheQueryIndexSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheQueryLoadSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheQueryMultiThreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheQueryOffheapMultiThreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheSqlQueryMultiThreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheUpdateSqlQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCrossCachesJoinsQueryTest;
+import org.apache.ignite.internal.processors.cache.SqlFieldsQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicFieldsQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicNearEnabledFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicNearEnabledQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheClientQueryReplicatedNodeRestartSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheDistributedQueryCancelSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedFieldsQueryP2PEnabledSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedQueryP2PDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedSnapshotEnabledQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNoRebalanceSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeFailTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryAbstractDistributedJoinSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest2;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNoRebalanceSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQueryP2PEnabledSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQueryP2PDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalAtomicQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQueryCancelOrTimeoutSelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryTransformerSelfTest;
+import org.apache.ignite.internal.processors.cache.query.GridCacheSwapScanQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.query.IndexingSpiQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.query.IndexingSpiQueryTxSelfTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlSchemaIndexingTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlSplitterSelfTest;
+import org.apache.ignite.internal.processors.query.h2.GridH2IndexRebuildTest;
+import org.apache.ignite.internal.processors.query.h2.GridH2IndexingInMemSelfTest;
+import org.apache.ignite.internal.processors.query.h2.GridH2IndexingOffheapSelfTest;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2TableSelfTest;
+import org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest;
 import org.apache.ignite.internal.processors.query.h2.sql.GridQueryParsingTest;
+import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryTest;
+import org.apache.ignite.spi.communication.tcp.GridOrderedMessageCancelSelfTest;
 import org.apache.ignite.testframework.IgniteTestSuite;
 
 /**
@@ -93,6 +117,12 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
     public static TestSuite suite() throws Exception {
         IgniteTestSuite suite = new IgniteTestSuite("Ignite Cache Queries Test Suite");
 
+        // H2 tests.
+        suite.addTest(new TestSuite(GridH2TableSelfTest.class));
+        suite.addTest(new TestSuite(GridH2IndexingInMemSelfTest.class));
+        suite.addTest(new TestSuite(GridH2IndexingOffheapSelfTest.class));
+        suite.addTest(new TestSuite(GridH2IndexRebuildTest.class));
+
         // Parsing
         suite.addTestSuite(GridQueryParsingTest.class);
 
@@ -113,25 +143,17 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheAtomicQuerySelfTest.class);
         suite.addTestSuite(IgniteCacheAtomicNearEnabledQuerySelfTest.class);
         suite.addTestSuite(IgniteCachePartitionedQueryP2PDisabledSelfTest.class);
-        suite.addTestSuite(IgniteCachePartitionedQueryMultiThreadedSelfTest.class);
+
         suite.addTestSuite(IgniteCacheQueryIndexSelfTest.class);
         suite.addTestSuite(IgniteCacheCollocatedQuerySelfTest.class);
         suite.addTestSuite(IgniteCacheLargeResultSelfTest.class);
         suite.addTestSuite(GridCacheQueryInternalKeysSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryMultiThreadedSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryOffheapMultiThreadedSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class);
+
         suite.addTestSuite(IgniteCacheOffheapEvictQueryTest.class);
-        suite.addTestSuite(IgniteCacheSqlQueryMultiThreadedSelfTest.class);
-        suite.addTestSuite(IgniteCacheOffheapTieredMultithreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheOffheapIndexScanTest.class);
-        suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest2.class);
+
         suite.addTestSuite(IgniteCacheQueryAbstractDistributedJoinSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryNodeFailTest.class);
-        suite.addTestSuite(IgniteCacheClientQueryReplicatedNodeRestartSelfTest.class);
+
         suite.addTestSuite(GridCacheCrossCacheQuerySelfTest.class);
         suite.addTestSuite(GridCacheQuerySerializationSelfTest.class);
         suite.addTestSuite(IgniteBinaryObjectFieldsQuerySelfTest.class);
@@ -149,7 +171,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheJoinPartitionedAndReplicatedTest.class);
         suite.addTestSuite(IgniteCacheDistributedJoinNoIndexTest.class);
         suite.addTestSuite(IgniteCrossCachesJoinsQueryTest.class);
-        suite.addTestSuite(IgniteCacheCrossCacheJoinRandomTest.class);
+
         suite.addTestSuite(IgniteCacheDistributedJoinCustomAffinityMapper.class);
 
         suite.addTestSuite(IgniteCacheMergeSqlQuerySelfTest.class);
@@ -165,6 +187,58 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IndexingSpiQuerySelfTest.class);
         suite.addTestSuite(IndexingSpiQueryTxSelfTest.class);
 
+        // Fields queries.
+        suite.addTestSuite(SqlFieldsQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheLocalFieldsQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheReplicatedFieldsQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheReplicatedFieldsQueryP2PEnabledSelfTest.class);
+        suite.addTestSuite(IgniteCachePartitionedFieldsQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheAtomicFieldsQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheAtomicNearEnabledFieldsQuerySelfTest.class);
+        suite.addTestSuite(IgniteCachePartitionedFieldsQueryP2PEnabledSelfTest.class);
+        suite.addTestSuite(IgniteCacheFieldsQueryNoDataSelfTest.class);
+
+        suite.addTestSuite(GridCacheQueryIndexingDisabledSelfTest.class);
+
+        suite.addTestSuite(GridCacheSwapScanQuerySelfTest.class);
+
+        suite.addTestSuite(GridOrderedMessageCancelSelfTest.class);
+
+        suite.addTestSuite(CacheQueryOffheapEvictDataLostTest.class);
+
+        // Ignite cache and H2 comparison.
+        suite.addTestSuite(BaseH2CompareQueryTest.class);
+        suite.addTestSuite(H2CompareBigQueryTest.class);
+
+        // Cache query metrics.
+        suite.addTestSuite(CacheLocalQueryMetricsSelfTest.class);
+        suite.addTestSuite(CachePartitionedQueryMetricsDistributedSelfTest.class);
+        suite.addTestSuite(CachePartitionedQueryMetricsLocalSelfTest.class);
+        suite.addTestSuite(CacheReplicatedQueryMetricsDistributedSelfTest.class);
+        suite.addTestSuite(CacheReplicatedQueryMetricsLocalSelfTest.class);
+
+        // Cache query metrics.
+        suite.addTestSuite(CacheLocalQueryDetailMetricsSelfTest.class);
+        suite.addTestSuite(CachePartitionedQueryDetailMetricsDistributedSelfTest.class);
+        suite.addTestSuite(CachePartitionedQueryDetailMetricsLocalSelfTest.class);
+        suite.addTestSuite(CacheReplicatedQueryDetailMetricsDistributedSelfTest.class);
+        suite.addTestSuite(CacheReplicatedQueryDetailMetricsLocalSelfTest.class);
+
+        // Unmarshalling query test.
+        suite.addTestSuite(IgniteCacheP2pUnmarshallingQueryErrorTest.class);
+        suite.addTestSuite(IgniteCacheNoClassQuerySelfTest.class);
+
+        // Cancellation.
+
+        suite.addTestSuite(IgniteCacheDistributedQueryCancelSelfTest.class);
+        suite.addTestSuite(IgniteCacheLocalQueryCancelOrTimeoutSelfTest.class);
+        suite.addTestSuite(IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.class);
+
+        // Other.
+        suite.addTestSuite(CacheQueryNewClientSelfTest.class);
+        suite.addTestSuite(CacheOffheapBatchIndexingSingleTypeTest.class);
+        suite.addTestSuite(CacheSqlQueryValueCopySelfTest.class);
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1e97d78c/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
index 8ac219f..a0db90c 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
@@ -18,41 +18,21 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
-import org.apache.ignite.internal.processors.cache.CacheLocalQueryDetailMetricsSelfTest;
-import org.apache.ignite.internal.processors.cache.CacheLocalQueryMetricsSelfTest;
-import org.apache.ignite.internal.processors.cache.CacheOffheapBatchIndexingSingleTypeTest;
-import org.apache.ignite.internal.processors.cache.CachePartitionedQueryDetailMetricsDistributedSelfTest;
-import org.apache.ignite.internal.processors.cache.CachePartitionedQueryDetailMetricsLocalSelfTest;
-import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsDistributedSelfTest;
-import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsLocalSelfTest;
-import org.apache.ignite.internal.processors.cache.CacheQueryNewClientSelfTest;
-import org.apache.ignite.internal.processors.cache.CacheQueryOffheapEvictDataLostTest;
-import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryDetailMetricsDistributedSelfTest;
-import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryDetailMetricsLocalSelfTest;
-import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsDistributedSelfTest;
-import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsLocalSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheScanPartitionQueryFallbackSelfTest;
-import org.apache.ignite.internal.processors.cache.CacheSqlQueryValueCopySelfTest;
-import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexingDisabledSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheFieldsQueryNoDataSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheNoClassQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingQueryErrorTest;
-import org.apache.ignite.internal.processors.cache.SqlFieldsQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicFieldsQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicNearEnabledFieldsQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheDistributedQueryCancelSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedFieldsQueryP2PEnabledSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedFieldsQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheCrossCacheJoinRandomTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapTieredMultithreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCachePartitionedQueryMultiThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheQueryEvictsMultiThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheQueryMultiThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheQueryOffheapMultiThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheSqlQueryMultiThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheClientQueryReplicatedNodeRestartSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQueryP2PEnabledSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQueryCancelOrTimeoutSelfTest;
-import org.apache.ignite.internal.processors.cache.query.GridCacheSwapScanQuerySelfTest;
-import org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest;
-import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryTest;
-import org.apache.ignite.spi.communication.tcp.GridOrderedMessageCancelSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeFailTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest2;
 import org.apache.ignite.testframework.IgniteTestSuite;
 
 /**
@@ -66,60 +46,21 @@ public class IgniteCacheQuerySelfTestSuite2 extends TestSuite {
     public static TestSuite suite() throws Exception {
         TestSuite suite = new IgniteTestSuite("Ignite Cache Queries Test Suite 2");
 
-        // Scan queries.
+        suite.addTestSuite(IgniteCacheQueryMultiThreadedSelfTest.class);
+        suite.addTestSuite(IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.class);
+        suite.addTestSuite(IgniteCacheQueryOffheapMultiThreadedSelfTest.class);
+        suite.addTestSuite(IgniteCacheOffheapTieredMultithreadedSelfTest.class);
+        suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);
+        suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class);
+        suite.addTestSuite(IgniteCacheCrossCacheJoinRandomTest.class);
+        suite.addTestSuite(IgniteCacheClientQueryReplicatedNodeRestartSelfTest.class);
+        suite.addTestSuite(IgniteCacheQueryNodeFailTest.class);
+        suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest.class);
+        suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest2.class);
+        suite.addTestSuite(IgniteCacheSqlQueryMultiThreadedSelfTest.class);
+        suite.addTestSuite(IgniteCachePartitionedQueryMultiThreadedSelfTest.class);
         suite.addTestSuite(CacheScanPartitionQueryFallbackSelfTest.class);
-
-        // Fields queries.
-        suite.addTestSuite(SqlFieldsQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheLocalFieldsQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheReplicatedFieldsQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheReplicatedFieldsQueryP2PEnabledSelfTest.class);
-        suite.addTestSuite(IgniteCachePartitionedFieldsQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheAtomicFieldsQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheAtomicNearEnabledFieldsQuerySelfTest.class);
-        suite.addTestSuite(IgniteCachePartitionedFieldsQueryP2PEnabledSelfTest.class);
-        suite.addTestSuite(IgniteCacheFieldsQueryNoDataSelfTest.class);
-
-        suite.addTestSuite(GridCacheQueryIndexingDisabledSelfTest.class);
-
-        suite.addTestSuite(GridCacheSwapScanQuerySelfTest.class);
-
-        suite.addTestSuite(GridOrderedMessageCancelSelfTest.class);
-
-        suite.addTestSuite(CacheQueryOffheapEvictDataLostTest.class);
-
-        // Ignite cache and H2 comparison.
-        suite.addTestSuite(BaseH2CompareQueryTest.class);
-        suite.addTestSuite(H2CompareBigQueryTest.class);
-
-        // Cache query metrics.
-        suite.addTestSuite(CacheLocalQueryMetricsSelfTest.class);
-        suite.addTestSuite(CachePartitionedQueryMetricsDistributedSelfTest.class);
-        suite.addTestSuite(CachePartitionedQueryMetricsLocalSelfTest.class);
-        suite.addTestSuite(CacheReplicatedQueryMetricsDistributedSelfTest.class);
-        suite.addTestSuite(CacheReplicatedQueryMetricsLocalSelfTest.class);
-
-        // Cache query metrics.
-        suite.addTestSuite(CacheLocalQueryDetailMetricsSelfTest.class);
-        suite.addTestSuite(CachePartitionedQueryDetailMetricsDistributedSelfTest.class);
-        suite.addTestSuite(CachePartitionedQueryDetailMetricsLocalSelfTest.class);
-        suite.addTestSuite(CacheReplicatedQueryDetailMetricsDistributedSelfTest.class);
-        suite.addTestSuite(CacheReplicatedQueryDetailMetricsLocalSelfTest.class);
-
-        // Unmarshalling query test.
-        suite.addTestSuite(IgniteCacheP2pUnmarshallingQueryErrorTest.class);
-        suite.addTestSuite(IgniteCacheNoClassQuerySelfTest.class);
-
-        // Cancellation.
         suite.addTestSuite(IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.class);
-        suite.addTestSuite(IgniteCacheDistributedQueryCancelSelfTest.class);
-        suite.addTestSuite(IgniteCacheLocalQueryCancelOrTimeoutSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.class);
-
-        // Other.
-        suite.addTestSuite(CacheQueryNewClientSelfTest.class);
-        suite.addTestSuite(CacheOffheapBatchIndexingSingleTypeTest.class);
-        suite.addTestSuite(CacheSqlQueryValueCopySelfTest.class);
 
         return suite;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1e97d78c/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteH2IndexingSpiTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteH2IndexingSpiTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteH2IndexingSpiTestSuite.java
index 18563be..ae3ab49 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteH2IndexingSpiTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteH2IndexingSpiTestSuite.java
@@ -18,10 +18,6 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
-import org.apache.ignite.internal.processors.query.h2.GridH2IndexRebuildTest;
-import org.apache.ignite.internal.processors.query.h2.GridH2IndexingInMemSelfTest;
-import org.apache.ignite.internal.processors.query.h2.GridH2IndexingOffheapSelfTest;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2TableSelfTest;
 
 /**
  * H2 indexing SPI tests.
@@ -34,17 +30,7 @@ public class IgniteH2IndexingSpiTestSuite extends TestSuite {
     public static TestSuite suite() throws Exception {
         TestSuite suite = new TestSuite("H2 Indexing SPI Test Suite");
 
-        // H2 Optimized table test.
-        suite.addTest(new TestSuite(GridH2TableSelfTest.class));
-
-        // H2 Indexing in-memory.
-        suite.addTest(new TestSuite(GridH2IndexingInMemSelfTest.class));
-
-        // H2 Off-heap memory.
-        suite.addTest(new TestSuite(GridH2IndexingOffheapSelfTest.class));
-
-        // Index rebuilding.
-        suite.addTest(new TestSuite(GridH2IndexRebuildTest.class));
+        // No-op.
 
         return suite;
     }


[03/16] ignite git commit: ignite-2412 Do not call 'asyncOp' for synchronous operations

Posted by vo...@apache.org.
ignite-2412 Do not call 'asyncOp' for synchronous operations


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

Branch: refs/heads/ignite-2.0
Commit: c530d47b8e3fd514e49bc59a1a7a3bde1060912a
Parents: 0c782b0
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 16 19:23:29 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 16 19:23:29 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      | 721 +++++++------------
 .../processors/cache/IgniteCacheProxy.java      |   8 -
 .../dht/atomic/GridDhtAtomicCache.java          | 472 +++++++-----
 .../dht/colocated/GridDhtColocatedCache.java    |  13 -
 .../distributed/near/GridNearAtomicCache.java   |  65 +-
 .../local/atomic/GridLocalAtomicCache.java      | 177 +----
 ...nabledMultiNodeLongTxTimeoutFullApiTest.java |   2 +-
 ...lockMessageSystemPoolStarvationSelfTest.java |  14 +-
 8 files changed, 597 insertions(+), 875 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c530d47b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 88aa4e0..5707b49 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -248,16 +248,16 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** Grid configuration. */
     @GridToStringExclude
-    protected IgniteConfiguration gridCfg;
+    private IgniteConfiguration gridCfg;
 
     /** Cache metrics. */
     protected CacheMetricsImpl metrics;
 
     /** Cache localMxBean. */
-    protected CacheMetricsMXBean localMxBean;
+    private CacheMetricsMXBean locMxBean;
 
     /** Cache mxBean. */
-    protected CacheMetricsMXBean clusterMxBean;
+    private CacheMetricsMXBean clusterMxBean;
 
     /** Logger. */
     protected IgniteLogger log;
@@ -288,9 +288,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     /** Asynchronous operations limit semaphore. */
     private Semaphore asyncOpsSem;
 
-    /** */
-    protected volatile boolean asyncToggled;
-
     /** {@inheritDoc} */
     @Override public String name() {
         return cacheCfg.getName();
@@ -334,7 +331,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         metrics = new CacheMetricsImpl(ctx);
 
-        localMxBean = new CacheLocalMetricsMXBeanImpl(this);
+        locMxBean = new CacheLocalMetricsMXBeanImpl(this);
         clusterMxBean = new CacheClusterMetricsMXBeanImpl(this);
 
         FileSystemConfiguration[] igfsCfgs = gridCfg.getFileSystemConfiguration();
@@ -367,18 +364,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
-     * Toggles async flag if someone calls {@code withAsync()}
-     * on proxy and since that we have to properly handle all cache
-     * operations (sync and async) to put them in proper sequence.
-     *
-     * TODO: https://issues.apache.org/jira/browse/IGNITE-4393
-     */
-    void toggleAsync() {
-        if (!asyncToggled)
-            asyncToggled = true;
-    }
-
-    /**
      * Prints memory stats.
      */
     public void printMemoryStats() {
@@ -471,49 +456,49 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     public abstract GridCachePreloader preloader();
 
     /** {@inheritDoc} */
-    @Override public Affinity<K> affinity() {
+    @Override public final Affinity<K> affinity() {
         return aff;
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings({"unchecked", "RedundantCast"})
-    @Override public <K1, V1> IgniteInternalCache<K1, V1> cache() {
+    @Override public final <K1, V1> IgniteInternalCache<K1, V1> cache() {
         return (IgniteInternalCache<K1, V1>)this;
     }
 
     /** {@inheritDoc} */
-    @Override public GridCacheProxyImpl<K, V> forSubjectId(UUID subjId) {
+    @Override public final GridCacheProxyImpl<K, V> forSubjectId(UUID subjId) {
         CacheOperationContext opCtx = new CacheOperationContext(false, subjId, false, null, false, null);
 
         return new GridCacheProxyImpl<>(ctx, this, opCtx);
     }
 
     /** {@inheritDoc} */
-    @Override public boolean skipStore() {
+    @Override public final boolean skipStore() {
         return false;
     }
 
     /** {@inheritDoc} */
-    @Override public GridCacheProxyImpl<K, V> setSkipStore(boolean skipStore) {
+    @Override public final GridCacheProxyImpl<K, V> setSkipStore(boolean skipStore) {
         CacheOperationContext opCtx = new CacheOperationContext(true, null, false, null, false, null);
 
         return new GridCacheProxyImpl<>(ctx, this, opCtx);
     }
 
     /** {@inheritDoc} */
-    @Override public <K1, V1> GridCacheProxyImpl<K1, V1> keepBinary() {
+    @Override public final <K1, V1> GridCacheProxyImpl<K1, V1> keepBinary() {
         CacheOperationContext opCtx = new CacheOperationContext(false, null, true, null, false, null);
 
         return new GridCacheProxyImpl<>((GridCacheContext<K1, V1>)ctx, (GridCacheAdapter<K1, V1>)this, opCtx);
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public ExpiryPolicy expiry() {
+    @Nullable @Override public final ExpiryPolicy expiry() {
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public GridCacheProxyImpl<K, V> withExpiryPolicy(ExpiryPolicy plc) {
+    @Override public final GridCacheProxyImpl<K, V> withExpiryPolicy(ExpiryPolicy plc) {
         assert !CU.isUtilityCache(ctx.name());
         assert !CU.isAtomicsCache(ctx.name());
         assert !CU.isMarshallerCache(ctx.name());
@@ -524,14 +509,14 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalCache<K, V> withNoRetries() {
+    @Override public final IgniteInternalCache<K, V> withNoRetries() {
         CacheOperationContext opCtx = new CacheOperationContext(false, null, false, null, true, null);
 
         return new GridCacheProxyImpl<>(ctx, this, opCtx);
     }
 
     /** {@inheritDoc} */
-    @Override public CacheConfiguration configuration() {
+    @Override public final CacheConfiguration configuration() {
         return ctx.config();
     }
 
@@ -630,7 +615,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isEmpty() {
+    @Override public final boolean isEmpty() {
         try {
             return localSize(CachePeekModes.ONHEAP_ONLY) == 0;
         }
@@ -640,7 +625,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public boolean containsKey(K key) {
+    @Override public final boolean containsKey(K key) {
         try {
             return containsKeyAsync(key).get();
         }
@@ -667,7 +652,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public boolean containsKeys(Collection<? extends K> keys) {
+    @Override public final boolean containsKeys(Collection<? extends K> keys) {
         try {
             return containsKeysAsync(keys).get();
         }
@@ -677,7 +662,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> containsKeysAsync(final Collection<? extends K> keys) {
+    @Override public final IgniteInternalFuture<Boolean> containsKeysAsync(final Collection<? extends K> keys) {
         A.notNull(keys, "keys");
 
         return getAllAsync(
@@ -708,7 +693,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public Iterable<Cache.Entry<K, V>> localEntries(CachePeekMode[] peekModes) throws IgniteCheckedException {
+    @Override public final Iterable<Cache.Entry<K, V>> localEntries(CachePeekMode[] peekModes) throws IgniteCheckedException {
         assert peekModes != null;
 
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
@@ -765,7 +750,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @SuppressWarnings("ForLoopReplaceableByForEach")
-    @Nullable @Override public V localPeek(K key,
+    @Nullable @Override public final V localPeek(K key,
         CachePeekMode[] peekModes,
         @Nullable IgniteCacheExpiryPolicy plc)
         throws IgniteCheckedException {
@@ -907,7 +892,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      *
      * @param ldr Class loader to undeploy.
      */
-    public void onUndeploy(ClassLoader ldr) {
+    public final void onUndeploy(ClassLoader ldr) {
         ctx.deploy().onUndeploy(ldr, context());
     }
 
@@ -916,7 +901,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param key Entry key.
      * @return Entry or <tt>null</tt>.
      */
-    @Nullable public GridCacheEntryEx peekEx(KeyCacheObject key) {
+    @Nullable public final GridCacheEntryEx peekEx(KeyCacheObject key) {
         return entry0(key, ctx.affinity().affinityTopologyVersion(), false, false);
     }
 
@@ -925,7 +910,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param key Entry key.
      * @return Entry or <tt>null</tt>.
      */
-    @Nullable public GridCacheEntryEx peekEx(Object key) {
+    @Nullable public final GridCacheEntryEx peekEx(Object key) {
         return entry0(ctx.toCacheKeyObject(key), ctx.affinity().affinityTopologyVersion(), false, false);
     }
 
@@ -933,7 +918,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param key Entry key.
      * @return Entry (never {@code null}).
      */
-    public GridCacheEntryEx entryEx(Object key) {
+    public final GridCacheEntryEx entryEx(Object key) {
         return entryEx(ctx.toCacheKeyObject(key), false);
     }
 
@@ -941,7 +926,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param key Entry key.
      * @return Entry (never {@code null}).
      */
-    public GridCacheEntryEx entryEx(KeyCacheObject key) {
+    public final GridCacheEntryEx entryEx(KeyCacheObject key) {
         return entryEx(key, false);
     }
 
@@ -996,24 +981,24 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     /**
      * @return Set of internal cached entry representations.
      */
-    public Iterable<? extends GridCacheEntryEx> entries() {
+    public final Iterable<? extends GridCacheEntryEx> entries() {
         return allEntries();
     }
 
     /**
      * @return Set of internal cached entry representations.
      */
-    public Iterable<? extends GridCacheEntryEx> allEntries() {
+    public final Iterable<? extends GridCacheEntryEx> allEntries() {
         return map.entries();
     }
 
     /** {@inheritDoc} */
-    @Override public Set<Cache.Entry<K, V>> entrySet() {
+    @Override public final Set<Cache.Entry<K, V>> entrySet() {
         return entrySet((CacheEntryPredicate[])null);
     }
 
     /** {@inheritDoc} */
-    @Override public Set<Cache.Entry<K, V>> entrySetx(final CacheEntryPredicate... filter) {
+    @Override public final Set<Cache.Entry<K, V>> entrySetx(final CacheEntryPredicate... filter) {
         boolean keepBinary = ctx.keepBinary();
 
         return new EntrySet(map.entrySet(filter), keepBinary);
@@ -1025,17 +1010,17 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public Set<K> keySet() {
+    @Override public final Set<K> keySet() {
         return new KeySet(map.entrySet());
     }
 
     /** {@inheritDoc} */
-    @Override public Set<K> keySetx() {
+    @Override public final Set<K> keySetx() {
         return keySet();
     }
 
     /** {@inheritDoc} */
-    @Override public Set<K> primaryKeySet() {
+    @Override public final Set<K> primaryKeySet() {
         return new KeySet(map.entrySet(CU.cachePrimary(ctx.grid().affinity(ctx.name()), ctx.localNode())));
     }
 
@@ -1057,7 +1042,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param filter Filters.
      * @return Collection of cached values.
      */
-    public Iterable<V> values(final CacheEntryPredicate... filter) {
+    public final Iterable<V> values(final CacheEntryPredicate... filter) {
         return new Iterable<V>() {
             @Override public Iterator<V> iterator() {
                 return new Iterator<V>() {
@@ -1083,12 +1068,12 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      *
      * @param key Entry key.
      */
-    public void removeIfObsolete(KeyCacheObject key) {
+    public final void removeIfObsolete(KeyCacheObject key) {
         assert key != null;
 
         GridCacheMapEntry entry = map.getEntry(key);
 
-        if (entry.obsolete())
+        if (entry != null && entry.obsolete())
             removeEntry(entry);
     }
 
@@ -1272,11 +1257,11 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     /**
      * @param entry Removes entry from cache if currently mapped value is the same as passed.
      */
-    public void removeEntry(GridCacheEntryEx entry) {
-        boolean removed = map.removeEntry(entry);
+    public final void removeEntry(GridCacheEntryEx entry) {
+        boolean rmvd = map.removeEntry(entry);
 
         if (log.isDebugEnabled()) {
-            if (removed)
+            if (rmvd)
                 log.debug("Removed entry from cache: " + entry);
             else
                 log.debug("Remove will not be done for key (entry got replaced or removed): " + entry.key());
@@ -1311,7 +1296,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public V getForcePrimary(K key) throws IgniteCheckedException {
+    @Override public final V getForcePrimary(K key) throws IgniteCheckedException {
         String taskName = ctx.kernalContext().job().currentTaskName();
 
         return getAllAsync(
@@ -1328,7 +1313,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<V> getForcePrimaryAsync(final K key) {
+    @Override public final IgniteInternalFuture<V> getForcePrimaryAsync(final K key) {
         String taskName = ctx.kernalContext().job().currentTaskName();
 
         return getAllAsync(
@@ -1349,7 +1334,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    public V getTopologySafe(K key) throws IgniteCheckedException {
+    public final V getTopologySafe(K key) throws IgniteCheckedException {
         String taskName = ctx.kernalContext().job().currentTaskName();
 
         return getAllAsync(
@@ -1366,12 +1351,12 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public Map<K, V> getAllOutTx(Set<? extends K> keys) throws IgniteCheckedException {
+    @Nullable @Override public final Map<K, V> getAllOutTx(Set<? extends K> keys) throws IgniteCheckedException {
         return getAllOutTxAsync(keys).get();
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Map<K, V>> getAllOutTxAsync(Set<? extends K> keys) {
+    @Override public final IgniteInternalFuture<Map<K, V>> getAllOutTxAsync(Set<? extends K> keys) {
         String taskName = ctx.kernalContext().job().currentTaskName();
 
         return getAllAsync(keys,
@@ -1385,15 +1370,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             false);
     }
 
-    /**
-     * @param key Key.
-     * @param topVer Topology version.
-     * @return Entry.
-     */
-    @Nullable protected GridCacheEntryEx entryExSafe(KeyCacheObject key, AffinityTopologyVersion topVer) {
-        return entryEx(key);
-    }
-
     /** {@inheritDoc} */
     @Nullable @Override public V get(K key) throws IgniteCheckedException {
         A.notNull(key, "key");
@@ -1533,14 +1509,14 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public Map<K, V> getAll(@Nullable Collection<? extends K> keys) throws IgniteCheckedException {
+    @Override public final Map<K, V> getAll(@Nullable Collection<? extends K> keys) throws IgniteCheckedException {
         A.notNull(keys, "keys");
 
         boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         long start = statsEnabled ? System.nanoTime() : 0L;
 
-        Map<K, V> map = getAll(keys, !ctx.keepBinary(), false);
+        Map<K, V> map = getAll0(keys, !ctx.keepBinary(), false);
 
         if (ctx.config().getInterceptor() != null)
             map = interceptGet(keys, map);
@@ -1560,7 +1536,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         long start = statsEnabled ? System.nanoTime() : 0L;
 
-        Map<K, T2<V, GridCacheVersion>> map = (Map<K, T2<V, GridCacheVersion>>)getAll(keys, !ctx.keepBinary(), true);
+        Map<K, T2<V, GridCacheVersion>> map = (Map<K, T2<V, GridCacheVersion>>)getAll0(keys, !ctx.keepBinary(), true);
 
         Collection<CacheEntry<K, V>> res = new HashSet<>();
 
@@ -1875,7 +1851,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param needVer If {@code true} returns values as tuples containing value and version.
      * @return Future.
      */
-    public final <K1, V1> IgniteInternalFuture<Map<K1, V1>> getAllAsync0(
+    protected final <K1, V1> IgniteInternalFuture<Map<K1, V1>> getAllAsync0(
         @Nullable final Collection<KeyCacheObject> keys,
         final boolean readThrough,
         boolean checkTx,
@@ -2141,7 +2117,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public V getAndPut(K key, V val) throws IgniteCheckedException {
+    @Override public final V getAndPut(K key, V val) throws IgniteCheckedException {
         return getAndPut(key, val, null);
     }
 
@@ -2163,7 +2139,24 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        V prevVal = syncOp(new SyncOp<V>(true) {
+        V prevVal = getAndPut0(key, val, filter);
+
+        if (statsEnabled)
+            metrics0().addPutAndGetTimeNanos(System.nanoTime() - start);
+
+        return prevVal;
+    }
+
+    /**
+     * @param key Key.
+     * @param val Value.
+     * @param filter Optional filter.
+     * @return Previous value.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected V getAndPut0(final K key, final V val, @Nullable final CacheEntryPredicate filter)
+        throws IgniteCheckedException {
+        return syncOp(new SyncOp<V>(true) {
             @Override public V op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
                 return (V)tx.putAsync(ctx, null, key, val, true, filter).get().value();
             }
@@ -2172,15 +2165,10 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 return "put [key=" + key + ", val=" + val + ", filter=" + filter + ']';
             }
         });
-
-        if (statsEnabled)
-            metrics0().addPutAndGetTimeNanos(System.nanoTime() - start);
-
-        return prevVal;
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<V> getAndPutAsync(K key, V val) {
+    @Override public final IgniteInternalFuture<V> getAndPutAsync(K key, V val) {
         return getAndPutAsync(key, val, null);
     }
 
@@ -2190,11 +2178,16 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param filter Filter.
      * @return Put operation future.
      */
-    public IgniteInternalFuture<V> getAndPutAsync(K key, V val, @Nullable CacheEntryPredicate filter) {
+    protected final IgniteInternalFuture<V> getAndPutAsync(K key, V val, @Nullable CacheEntryPredicate filter) {
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
 
+        A.notNull(key, "key", val, "val");
+
+        if (keyCheck)
+            validateCacheKey(key);
+
         IgniteInternalFuture<V> fut = getAndPutAsync0(key, val, filter);
 
         if (statsEnabled)
@@ -2209,13 +2202,10 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param filter Optional filter.
      * @return Put operation future.
      */
-    public IgniteInternalFuture<V> getAndPutAsync0(final K key, final V val,
-        @Nullable final CacheEntryPredicate filter) {
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
+    public IgniteInternalFuture<V> getAndPutAsync0(final K key,
+        final V val,
+        @Nullable final CacheEntryPredicate filter)
+    {
         return asyncOp(new AsyncOp<V>() {
             @Override public IgniteInternalFuture<V> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
                 return tx.putAsync(ctx, readyTopVer, key, val, true, filter)
@@ -2229,7 +2219,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public boolean put(final K key, final V val) throws IgniteCheckedException {
+    @Override public final boolean put(final K key, final V val) throws IgniteCheckedException {
         return put(key, val, null);
     }
 
@@ -2253,7 +2243,26 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        Boolean stored = syncOp(new SyncOp<Boolean>(true) {
+        boolean stored = put0(key, val, filter);
+
+        if (statsEnabled && stored)
+            metrics0().addPutTimeNanos(System.nanoTime() - start);
+
+        return stored;
+    }
+
+    /**
+     * @param key Key.
+     * @param val Value.
+     * @param filter Filter.
+     * @return {@code True} if optional filter passed and value was stored in cache,
+     *      {@code false} otherwise. Note that this method will return {@code true} if filter is not
+     *      specified.
+     * @throws IgniteCheckedException If put operation failed.
+     */
+    protected boolean put0(final K key, final V val, final CacheEntryPredicate filter)
+        throws IgniteCheckedException {
+        Boolean res = syncOp(new SyncOp<Boolean>(true) {
             @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
                 return tx.putAsync(ctx, null, key, val, false, filter).get().success();
             }
@@ -2263,10 +2272,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             }
         });
 
-        if (statsEnabled)
-            metrics0().addPutTimeNanos(System.nanoTime() - start);
+        assert res != null;
 
-        return stored;
+        return res;
     }
 
     /** {@inheritDoc} */
@@ -2308,7 +2316,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public <T> EntryProcessorResult<T> invoke(@Nullable AffinityTopologyVersion topVer,
+    @Nullable @Override public final <T> EntryProcessorResult<T> invoke(@Nullable AffinityTopologyVersion topVer,
         K key,
         EntryProcessor<K, V, T> entryProcessor,
         Object... args) throws IgniteCheckedException {
@@ -2541,7 +2549,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> putAsync(K key, V val) {
+    @Override public final IgniteInternalFuture<Boolean> putAsync(K key, V val) {
         return putAsync(key, val, null);
     }
 
@@ -2551,9 +2559,12 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param filter Filter.
      * @return Put future.
      */
-    public IgniteInternalFuture<Boolean> putAsync(K key, V val, @Nullable CacheEntryPredicate filter) {
+    public final IgniteInternalFuture<Boolean> putAsync(K key, V val, @Nullable CacheEntryPredicate filter) {
         A.notNull(key, "key", val, "val");
 
+        if (keyCheck)
+            validateCacheKey(key);
+
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
@@ -2574,9 +2585,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      */
     public IgniteInternalFuture<Boolean> putAsync0(final K key, final V val,
         @Nullable final CacheEntryPredicate filter) {
-        if (keyCheck)
-            validateCacheKey(key);
-
         return asyncOp(new AsyncOp<Boolean>() {
             @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
                 return tx.putAsync(ctx,
@@ -2601,267 +2609,82 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public V getAndPutIfAbsent(final K key, final V val) throws IgniteCheckedException {
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        return syncOp(new SyncOp<V>(true) {
-            @Override public V op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
-                return (V)tx.putAsync(ctx, null, key, val, true, ctx.noVal()).get().value();
-            }
-
-            @Override public String toString() {
-                return "putIfAbsent [key=" + key + ", val=" + val + ']';
-            }
-        });
+    @Nullable @Override public final V getAndPutIfAbsent(final K key, final V val) throws IgniteCheckedException {
+        return getAndPut(key, val, ctx.noVal());
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<V> getAndPutIfAbsentAsync(final K key, final V val) {
-        final boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        IgniteInternalFuture<V> fut = asyncOp(new AsyncOp<V>() {
-            @Override public IgniteInternalFuture<V> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
-                return tx.putAsync(ctx, readyTopVer, key, val, true, ctx.noVal())
-                    .chain((IgniteClosure<IgniteInternalFuture<GridCacheReturn>, V>)RET2VAL);
-            }
-
-            @Override public String toString() {
-                return "putIfAbsentAsync [key=" + key + ", val=" + val + ']';
-            }
-        });
-
-        if (statsEnabled)
-            fut.listen(new UpdatePutTimeStatClosure<V>(metrics0(), start));
-
-        return fut;
+    @Override public final IgniteInternalFuture<V> getAndPutIfAbsentAsync(final K key, final V val) {
+        return getAndPutAsync(key, val, ctx.noVal());
     }
 
     /** {@inheritDoc} */
-    @Override public boolean putIfAbsent(final K key, final V val) throws IgniteCheckedException {
-        boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        long start = statsEnabled ? System.nanoTime() : 0L;
-
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        Boolean stored = syncOp(new SyncOp<Boolean>(true) {
-            @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
-                return tx.putAsync(ctx, null, key, val, false, ctx.noVal()).get().success();
-            }
-
-            @Override public String toString() {
-                return "putxIfAbsent [key=" + key + ", val=" + val + ']';
-            }
-        });
-
-        if (statsEnabled && stored)
-            metrics0().addPutTimeNanos(System.nanoTime() - start);
-
-        return stored;
+    @Override public final boolean putIfAbsent(final K key, final V val) throws IgniteCheckedException {
+        return put(key, val, ctx.noVal());
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> putIfAbsentAsync(final K key, final V val) {
-        final boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        IgniteInternalFuture<Boolean> fut = asyncOp(new AsyncOp<Boolean>() {
-            @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
-                return tx.putAsync(ctx,
-                    readyTopVer,
-                    key,
-                    val,
-                    false,
-                    ctx.noVal()).chain(
-                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn>, Boolean>)RET2FLAG);
-            }
-
-            @Override public String toString() {
-                return "putxIfAbsentAsync [key=" + key + ", val=" + val + ']';
-            }
-        });
-
-        if (statsEnabled)
-            fut.listen(new UpdatePutTimeStatClosure<Boolean>(metrics0(), start));
-
-        return fut;
+    @Override public final IgniteInternalFuture<Boolean> putIfAbsentAsync(final K key, final V val) {
+        return putAsync(key, val, ctx.noVal());
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public V getAndReplace(final K key, final V val) throws IgniteCheckedException {
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        return syncOp(new SyncOp<V>(true) {
-            @Override public V op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
-                return (V)tx.putAsync(ctx, null, key, val, true, ctx.hasVal()).get().value();
-            }
-
-            @Override public String toString() {
-                return "replace [key=" + key + ", val=" + val + ']';
-            }
-        });
+    @Nullable @Override public final V getAndReplace(final K key, final V val) throws IgniteCheckedException {
+        return getAndPut(key, val, ctx.hasVal());
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<V> getAndReplaceAsync(final K key, final V val) {
-        final boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        IgniteInternalFuture<V> fut = asyncOp(new AsyncOp<V>() {
-            @Override public IgniteInternalFuture<V> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
-                return tx.putAsync(ctx, readyTopVer, key, val, true, ctx.hasVal()).chain(
-                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn>, V>)RET2VAL);
-            }
-
-            @Override public String toString() {
-                return "replaceAsync [key=" + key + ", val=" + val + ']';
-            }
-        });
-
-        if (statsEnabled)
-            fut.listen(new UpdatePutAndGetTimeStatClosure<V>(metrics0(), start));
-
-        return fut;
+    @Override public final IgniteInternalFuture<V> getAndReplaceAsync(final K key, final V val) {
+        return getAndPutAsync(key, val, ctx.hasVal());
     }
 
     /** {@inheritDoc} */
-    @Override public boolean replace(final K key, final V val) throws IgniteCheckedException {
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        return syncOp(new SyncOp<Boolean>(true) {
-            @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
-                return tx.putAsync(ctx, null, key, val, false, ctx.hasVal()).get().success();
-            }
-
-            @Override public String toString() {
-                return "replacex [key=" + key + ", val=" + val + ']';
-            }
-        });
+    @Override public final boolean replace(final K key, final V val) throws IgniteCheckedException {
+        return put(key, val, ctx.hasVal());
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> replaceAsync(final K key, final V val) {
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        return asyncOp(new AsyncOp<Boolean>() {
-            @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
-                return tx.putAsync(ctx, readyTopVer, key, val, false, ctx.hasVal()).chain(
-                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn>, Boolean>) RET2FLAG);
-            }
-
-            @Override public String toString() {
-                return "replacexAsync [key=" + key + ", val=" + val + ']';
-            }
-        });
+    @Override public final IgniteInternalFuture<Boolean> replaceAsync(final K key, final V val) {
+        return putAsync(key, val, ctx.hasVal());
     }
 
     /** {@inheritDoc} */
-    @Override public boolean replace(final K key, final V oldVal, final V newVal) throws IgniteCheckedException {
-        A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal");
-
-        if (keyCheck)
-            validateCacheKey(key);
+    @Override public final boolean replace(final K key, final V oldVal, final V newVal) throws IgniteCheckedException {
+        A.notNull(oldVal, "oldVal");
 
-        return syncOp(new SyncOp<Boolean>(true) {
-            @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
-                // Register before hiding in the filter.
-                if (ctx.deploymentEnabled())
-                    ctx.deploy().registerClass(oldVal);
-
-                return tx.putAsync(ctx, null, key, newVal, false, ctx.equalsVal(oldVal)).get()
-                    .success();
-            }
-
-            @Override public String toString() {
-                return "replace [key=" + key + ", oldVal=" + oldVal + ", newVal=" + newVal + ']';
-            }
-        });
+        return put(key, newVal, ctx.equalsVal(oldVal));
     }
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<Boolean> replaceAsync(final K key, final V oldVal, final V newVal) {
-        final boolean statsEnabled = ctx.config().isStatisticsEnabled();
+        A.notNull(oldVal, "oldVal");
 
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
-        A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        IgniteInternalFuture<Boolean> fut = asyncOp(new AsyncOp<Boolean>() {
-            @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
-                // Register before hiding in the filter.
-                if (ctx.deploymentEnabled()) {
-                    try {
-                        ctx.deploy().registerClass(oldVal);
-                    }
-                    catch (IgniteCheckedException e) {
-                        return new GridFinishedFuture<>(e);
-                    }
-                }
-
-                return tx.putAsync(ctx, readyTopVer, key, newVal, false, ctx.equalsVal(oldVal)).chain(
-                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn>, Boolean>)RET2FLAG);
-            }
-
-            @Override public String toString() {
-                return "replaceAsync [key=" + key + ", oldVal=" + oldVal + ", newVal=" + newVal + ']';
-            }
-        });
-
-        if (statsEnabled)
-            fut.listen(new UpdatePutAndGetTimeStatClosure<Boolean>(metrics0(), start));
-
-        return fut;
+        return putAsync(key, newVal, ctx.equalsVal(oldVal));
     }
 
     /** {@inheritDoc} */
     @Override public void putAll(@Nullable final Map<? extends K, ? extends V> m) throws IgniteCheckedException {
+        if (F.isEmpty(m))
+            return;
+
         boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         long start = statsEnabled ? System.nanoTime() : 0L;
 
-        if (F.isEmpty(m))
-            return;
-
         if (keyCheck)
             validateCacheKeys(m.keySet());
 
+        putAll0(m);
+
+        if (statsEnabled)
+            metrics0().addPutTimeNanos(System.nanoTime() - start);
+    }
+
+    /**
+     * @param m Map.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected void putAll0(final Map<? extends K, ? extends V> m) throws IgniteCheckedException {
         syncOp(new SyncInOp(m.size() == 1) {
             @Override public void inOp(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
                 tx.putAllAsync(ctx, null, m, false).get();
@@ -2871,9 +2694,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 return "putAll [map=" + m + ']';
             }
         });
-
-        if (statsEnabled)
-            metrics0().addPutTimeNanos(System.nanoTime() - start);
     }
 
     /** {@inheritDoc} */
@@ -2884,6 +2704,14 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKeys(m.keySet());
 
+        return putAllAsync0(m);
+    }
+
+    /**
+     * @param m Map.
+     * @return Future.
+     */
+    protected IgniteInternalFuture<?> putAllAsync0(final Map<? extends K, ? extends V> m) {
         return asyncOp(new AsyncOp(m.keySet()) {
             @Override public IgniteInternalFuture<?> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
                 return tx.putAllAsync(ctx,
@@ -2909,11 +2737,25 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
+        V prevVal = getAndRemove0(key);
+
+        if (statsEnabled)
+            metrics0().addRemoveAndGetTimeNanos(System.nanoTime() - start);
+
+        return prevVal;
+    }
+
+    /**
+     * @param key Key.
+     * @return Previous value.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected V getAndRemove0(final K key) throws IgniteCheckedException {
         final boolean keepBinary = ctx.keepBinary();
 
-        V prevVal = syncOp(new SyncOp<V>(true) {
+        return syncOp(new SyncOp<V>(true) {
             @Override public V op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
-                K key0 = keepBinary ? (K)ctx.toCacheKeyObject(key) : key;
+                K key0 = keepBinary ? (K) ctx.toCacheKeyObject(key) : key;
 
                 V ret = tx.removeAllAsync(ctx,
                     null,
@@ -2923,9 +2765,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                     /*singleRmv*/false).get().value();
 
                 if (ctx.config().getInterceptor() != null) {
-                    K key = keepBinary ? (K)ctx.unwrapBinaryIfNeeded(key0, true, false) : key0;
+                    K key = keepBinary ? (K) ctx.unwrapBinaryIfNeeded(key0, true, false) : key0;
 
-                    return (V)ctx.config().getInterceptor().onBeforeRemove(new CacheEntryImpl(key, ret)).get2();
+                    return (V) ctx.config().getInterceptor().onBeforeRemove(new CacheEntryImpl(key, ret)).get2();
                 }
 
                 return ret;
@@ -2935,11 +2777,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 return "remove [key=" + key + ']';
             }
         });
-
-        if (statsEnabled)
-            metrics0().addRemoveAndGetTimeNanos(System.nanoTime() - start);
-
-        return prevVal;
     }
 
     /** {@inheritDoc} */
@@ -2953,7 +2790,20 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        IgniteInternalFuture<V> fut = asyncOp(new AsyncOp<V>() {
+        IgniteInternalFuture<V> fut = getAndRemoveAsync0(key);
+
+        if (statsEnabled)
+            fut.listen(new UpdateRemoveTimeStatClosure<V>(metrics0(), start));
+
+        return fut;
+    }
+
+    /**
+     * @param key Key.
+     * @return Future.
+     */
+    protected IgniteInternalFuture<V> getAndRemoveAsync0(final K key) {
+        return asyncOp(new AsyncOp<V>() {
             @Override public IgniteInternalFuture<V> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
                 // TODO should we invoke interceptor here?
                 return tx.removeAllAsync(ctx,
@@ -2968,11 +2818,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 return "removeAsync [key=" + key + ']';
             }
         });
-
-        if (statsEnabled)
-            fut.listen(new UpdateRemoveTimeStatClosure<V>(metrics0(), start));
-
-        return fut;
     }
 
     /** {@inheritDoc} */
@@ -3005,6 +2850,17 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKeys(keys);
 
+        removeAll0(keys);
+
+        if (statsEnabled)
+            metrics0().addRemoveTimeNanos(System.nanoTime() - start);
+    }
+
+    /**
+     * @param keys Keys.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected void removeAll0(final Collection<? extends K> keys) throws IgniteCheckedException {
         syncOp(new SyncInOp(keys.size() == 1) {
             @Override public void inOp(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
                 tx.removeAllAsync(ctx,
@@ -3019,24 +2875,34 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 return "removeAll [keys=" + keys + ']';
             }
         });
-
-        if (statsEnabled)
-            metrics0().addRemoveTimeNanos(System.nanoTime() - start);
     }
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> removeAllAsync(@Nullable final Collection<? extends K> keys) {
+        if (F.isEmpty(keys))
+            return new GridFinishedFuture<Object>();
+
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
 
-        if (F.isEmpty(keys))
-            return new GridFinishedFuture<Object>();
-
         if (keyCheck)
             validateCacheKeys(keys);
 
-        IgniteInternalFuture<Object> fut = asyncOp(new AsyncOp(keys) {
+        IgniteInternalFuture<Object> fut = removeAllAsync0(keys);
+
+        if (statsEnabled)
+            fut.listen(new UpdateRemoveTimeStatClosure<>(metrics0(), start));
+
+        return fut;
+    }
+
+    /**
+     * @param keys Keys.
+     * @return Future.
+     */
+    protected IgniteInternalFuture<Object> removeAllAsync0(final Collection<? extends K> keys) {
+        return asyncOp(new AsyncOp(keys) {
             @Override public IgniteInternalFuture<?> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
                 return tx.removeAllAsync(ctx,
                     readyTopVer,
@@ -3050,15 +2916,20 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 return "removeAllAsync [keys=" + keys + ']';
             }
         });
-
-        if (statsEnabled)
-            fut.listen(new UpdateRemoveTimeStatClosure<>(metrics0(), start));
-
-        return fut;
     }
 
     /** {@inheritDoc} */
     @Override public boolean remove(final K key) throws IgniteCheckedException {
+        return remove(key, (CacheEntryPredicate)null);
+    }
+
+    /**
+     * @param key Key.
+     * @param filter Filter.
+     * @return {@code True} if entry was removed.
+     * @throws IgniteCheckedException If failed.
+     */
+    public boolean remove(final K key, @Nullable CacheEntryPredicate filter) throws IgniteCheckedException {
         boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         long start = statsEnabled ? System.nanoTime() : 0L;
@@ -3068,13 +2939,27 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        boolean rmv = syncOp(new SyncOp<Boolean>(true) {
+        boolean rmv = remove0(key, filter);
+
+        if (statsEnabled && rmv)
+            metrics0().addRemoveTimeNanos(System.nanoTime() - start);
+
+        return rmv;
+    }
+
+    /**
+     * @param key Key.
+     * @return {@code True} if entry was removed.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected boolean remove0(final K key, final CacheEntryPredicate filter) throws IgniteCheckedException {
+        Boolean res = syncOp(new SyncOp<Boolean>(true) {
             @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
                 return tx.removeAllAsync(ctx,
                     null,
                     Collections.singletonList(key),
                     /*retval*/false,
-                    null,
+                    filter,
                     /*singleRmv*/true).get().success();
             }
 
@@ -3083,10 +2968,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             }
         });
 
-        if (statsEnabled && rmv)
-            metrics0().addRemoveTimeNanos(System.nanoTime() - start);
+        assert res != null;
 
-        return rmv;
+        return res;
     }
 
     /** {@inheritDoc} */
@@ -3111,7 +2995,21 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        IgniteInternalFuture<Boolean> fut = asyncOp(new AsyncOp<Boolean>() {
+        IgniteInternalFuture<Boolean> fut = removeAsync0(key, filter);
+
+        if (statsEnabled)
+            fut.listen(new UpdateRemoveTimeStatClosure<Boolean>(metrics0(), start));
+
+        return fut;
+    }
+
+    /**
+     * @param key Key.
+     * @param filter Filter.
+     * @return Future.
+     */
+    protected IgniteInternalFuture<Boolean> removeAsync0(final K key, @Nullable final CacheEntryPredicate filter) {
+        return asyncOp(new AsyncOp<Boolean>() {
             @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
                 return tx.removeAllAsync(ctx,
                     readyTopVer,
@@ -3126,11 +3024,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 return "removeAsync [key=" + key + ", filter=" + filter + ']';
             }
         });
-
-        if (statsEnabled)
-            fut.listen(new UpdateRemoveTimeStatClosure<Boolean>(metrics0(), start));
-
-        return fut;
     }
 
     /** {@inheritDoc} */
@@ -3172,86 +3065,21 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public boolean remove(final K key, final V val) throws IgniteCheckedException {
-        boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        long start = statsEnabled ? System.nanoTime() : 0L;
-
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        boolean rmv = syncOp(new SyncOp<Boolean>(true) {
-            @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
-                // Register before hiding in the filter.
-                if (ctx.deploymentEnabled())
-                    ctx.deploy().registerClass(val);
-
-                return tx.removeAllAsync(ctx,
-                    null,
-                    Collections.singletonList(key),
-                    /*retval*/false,
-                    ctx.equalsVal(val),
-                    /*singleRmv*/false).get().success();
-            }
-
-            @Override public String toString() {
-                return "remove [key=" + key + ", val=" + val + ']';
-            }
-        });
+    @Override public final boolean remove(final K key, final V val) throws IgniteCheckedException {
+        A.notNull(val, "val");
 
-        if (statsEnabled && rmv)
-            metrics0().addRemoveTimeNanos(System.nanoTime() - start);
-
-        return rmv;
+        return remove(key, ctx.equalsVal(val));
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> removeAsync(final K key, final V val) {
-        final boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        IgniteInternalFuture<Boolean> fut = asyncOp(new AsyncOp<Boolean>() {
-            @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
-                // Register before hiding in the filter.
-                if (ctx.deploymentEnabled()) {
-                    try {
-                        ctx.deploy().registerClass(val);
-                    }
-                    catch (IgniteCheckedException e) {
-                        return new GridFinishedFuture<>(e);
-                    }
-                }
-
-                return tx.removeAllAsync(ctx,
-                    readyTopVer,
-                    Collections.singletonList(key),
-                    /*retval*/false,
-                    ctx.equalsVal(val),
-                    /*singleRmv*/false).chain(
-                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn>, Boolean>)RET2FLAG);
-            }
+    @Override public final IgniteInternalFuture<Boolean> removeAsync(final K key, final V val) {
+        A.notNull(key, "val");
 
-            @Override public String toString() {
-                return "removeAsync [key=" + key + ", val=" + val + ']';
-            }
-        });
-
-        if (statsEnabled)
-            fut.listen(new UpdateRemoveTimeStatClosure<Boolean>(metrics0(), start));
-
-        return fut;
+        return removeAsync(key, ctx.equalsVal(val));
     }
 
     /** {@inheritDoc} */
-    @Override public CacheMetrics clusterMetrics() {
+    @Override public final CacheMetrics clusterMetrics() {
         return clusterMetrics(ctx.grid().cluster().forCacheNodes(ctx.name()));
     }
 
@@ -3280,7 +3108,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public CacheMetricsMXBean localMxBean() {
-        return localMxBean;
+        return locMxBean;
     }
 
     /** {@inheritDoc} */
@@ -4610,9 +4438,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @return Failed future if waiting was interrupted.
      */
     @Nullable protected <T> IgniteInternalFuture<T> asyncOpAcquire() {
-        if (!asyncToggled)
-            return null;
-
         try {
             if (asyncOpsSem != null)
                 asyncOpsSem.acquire();
@@ -4630,8 +4455,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     /**
      * Releases asynchronous operations permit, if limited.
      */
-    protected void asyncOpRelease() {
-        if (asyncOpsSem != null && asyncToggled)
+    private void asyncOpRelease() {
+        if (asyncOpsSem != null)
             asyncOpsSem.release();
     }
 
@@ -4794,12 +4619,10 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @return Cached value.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public V get(K key, boolean deserializeBinary, final boolean needVer) throws IgniteCheckedException {
-        checkJta();
-
+    @Nullable public final V get(K key, boolean deserializeBinary, final boolean needVer) throws IgniteCheckedException {
         String taskName = ctx.kernalContext().job().currentTaskName();
 
-        return get(key, taskName, deserializeBinary, needVer);
+        return get0(key, taskName, deserializeBinary, needVer);
     }
 
     /**
@@ -4810,11 +4633,13 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @return Cached value.
      * @throws IgniteCheckedException If failed.
      */
-    protected V get(
+    protected V get0(
         final K key,
         String taskName,
         boolean deserializeBinary,
         boolean needVer) throws IgniteCheckedException {
+        checkJta();
+
         try {
             return getAsync(key,
                 !ctx.config().isReadFromBackup(),
@@ -4868,7 +4693,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @return Map of cached values.
      * @throws IgniteCheckedException If read failed.
      */
-    public Map<K, V> getAll(Collection<? extends K> keys, boolean deserializeBinary,
+    protected Map<K, V> getAll0(Collection<? extends K> keys, boolean deserializeBinary,
         boolean needVer) throws IgniteCheckedException {
         checkJta();
 
@@ -4923,7 +4748,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param key Cache key.
      * @throws IllegalArgumentException If validation fails.
      */
-    protected void validateCacheKey(Object key) {
+    protected final void validateCacheKey(Object key) {
         if (keyCheck) {
             CU.validateCacheKey(key);
 
@@ -4938,7 +4763,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param keys Cache keys.
      * @throws IgniteException If validation fails.
      */
-    protected void validateCacheKeys(Iterable<?> keys) {
+    protected final void validateCacheKeys(Iterable<?> keys) {
         if (keys == null)
             return;
 
@@ -4959,7 +4784,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param deserializeBinary Deserialize binary flag.
      * @return Public API iterator.
      */
-    protected Iterator<Cache.Entry<K, V>> iterator(final Iterator<? extends GridCacheEntryEx> it,
+    protected final Iterator<Cache.Entry<K, V>> iterator(final Iterator<? extends GridCacheEntryEx> it,
         final boolean deserializeBinary) {
         return new Iterator<Cache.Entry<K, V>>() {
             {
@@ -5277,7 +5102,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param plc Explicitly specified expiry policy for cache operation.
      * @return Expiry policy wrapper.
      */
-    @Nullable public IgniteCacheExpiryPolicy expiryPolicy(@Nullable ExpiryPolicy plc) {
+    @Nullable public final IgniteCacheExpiryPolicy expiryPolicy(@Nullable ExpiryPolicy plc) {
         if (plc == null)
             plc = ctx.expiry();
 
@@ -5402,7 +5227,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
          * @param opCtx Operation context.
          * @return Operation future.
          */
-        protected IgniteInternalFuture<T> waitTopologyFuture(IgniteInternalFuture<?> topFut,
+        private IgniteInternalFuture<T> waitTopologyFuture(IgniteInternalFuture<?> topFut,
             final AffinityTopologyVersion topVer,
             final IgniteTxLocalAdapter tx,
             final CacheOperationContext opCtx) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/c530d47b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index 90898f9..f03a3b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -334,14 +334,6 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteCache<K, V> withAsync() {
-        if (delegate instanceof GridCacheAdapter)
-            ((GridCacheAdapter)delegate).toggleAsync();
-
-        return super.withAsync();
-    }
-
-    /** {@inheritDoc} */
     @Override public IgniteCache<K, V> withSkipStore() {
         return skipStore();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c530d47b/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 b291bd2..07b9dad 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
@@ -139,7 +139,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     private CI2<GridNearAtomicAbstractUpdateRequest, GridNearAtomicUpdateResponse> updateReplyClos;
 
     /** Pending */
-    private GridDeferredAckMessageSender deferredUpdateMessageSender;
+    private GridDeferredAckMessageSender deferredUpdateMsgSnd;
 
     /** */
     private GridNearAtomicCache<K, V> near;
@@ -174,6 +174,11 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
+    @Override protected void checkJta() throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean isDhtAtomic() {
         return true;
     }
@@ -235,7 +240,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @Override public void start() throws IgniteCheckedException {
         super.start();
 
-        deferredUpdateMessageSender = new GridDeferredAckMessageSender(ctx.time(), ctx.closures()) {
+        deferredUpdateMsgSnd = new GridDeferredAckMessageSender(ctx.time(), ctx.closures()) {
             @Override public int getTimeout() {
                 return DEFERRED_UPDATE_RESPONSE_TIMEOUT;
             }
@@ -447,7 +452,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @Override public void stop() {
-        deferredUpdateMessageSender.stop();
+        deferredUpdateMsgSnd.stop();
     }
 
     /**
@@ -463,7 +468,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override protected V get(K key, String taskName, boolean deserializeBinary, boolean needVer)
+    @Override protected V get0(K key, String taskName, boolean deserializeBinary, boolean needVer)
         throws IgniteCheckedException {
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
 
@@ -540,6 +545,21 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
+    @Override protected Map<K, V> getAll0(Collection<? extends K> keys, boolean deserializeBinary, boolean needVer)
+        throws IgniteCheckedException {
+        return getAllAsyncInternal(keys,
+            !ctx.config().isReadFromBackup(),
+            true,
+            null,
+            ctx.kernalContext().job().currentTaskName(),
+            deserializeBinary,
+            false,
+            true,
+            needVer,
+            false).get();
+    }
+
+    /** {@inheritDoc} */
     @Override public IgniteInternalFuture<Map<K, V>> getAllAsync(
         @Nullable final Collection<? extends K> keys,
         final boolean forcePrimary,
@@ -551,6 +571,43 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         final boolean canRemap,
         final boolean needVer
     ) {
+        return getAllAsyncInternal(keys,
+            forcePrimary,
+            skipTx,
+            subjId,
+            taskName,
+            deserializeBinary,
+            skipVals,
+            canRemap,
+            needVer,
+            true);
+    }
+
+    /**
+     * @param keys Keys.
+     * @param forcePrimary Force primary flag.
+     * @param skipTx Skip tx flag.
+     * @param subjId Subject ID.
+     * @param taskName Task name.
+     * @param deserializeBinary Deserialize binary flag.
+     * @param skipVals Skip values flag.
+     * @param canRemap Can remap flag.
+     * @param needVer Need version flag.
+     * @param asyncOp Async operation flag.
+     * @return Future.
+     */
+    private IgniteInternalFuture<Map<K, V>> getAllAsyncInternal(
+        @Nullable final Collection<? extends K> keys,
+        final boolean forcePrimary,
+        boolean skipTx,
+        @Nullable UUID subjId,
+        final String taskName,
+        final boolean deserializeBinary,
+        final boolean skipVals,
+        final boolean canRemap,
+        final boolean needVer,
+        boolean asyncOp
+    ) {
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
 
         if (F.isEmpty(keys))
@@ -561,7 +618,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
-        subjId = ctx.subjectIdPerCall(null, opCtx);
+        subjId = ctx.subjectIdPerCall(subjId, opCtx);
 
         final UUID subjId0 = subjId;
 
@@ -569,57 +626,91 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
         final boolean skipStore = opCtx != null && opCtx.skipStore();
 
-        return asyncOp(new CO<IgniteInternalFuture<Map<K, V>>>() {
-            @Override public IgniteInternalFuture<Map<K, V>> apply() {
-                return getAllAsync0(ctx.cacheKeysView(keys),
-                    forcePrimary,
-                    subjId0,
-                    taskName,
-                    deserializeBinary,
-                    expiryPlc,
-                    skipVals,
-                    skipStore,
-                    canRemap,
-                    needVer);
-            }
-        });
+        if (asyncOp) {
+            return asyncOp(new CO<IgniteInternalFuture<Map<K, V>>>() {
+                @Override public IgniteInternalFuture<Map<K, V>> apply() {
+                    return getAllAsync0(ctx.cacheKeysView(keys),
+                        forcePrimary,
+                        subjId0,
+                        taskName,
+                        deserializeBinary,
+                        expiryPlc,
+                        skipVals,
+                        skipStore,
+                        canRemap,
+                        needVer);
+                }
+            });
+        }
+        else {
+            return getAllAsync0(ctx.cacheKeysView(keys),
+                forcePrimary,
+                subjId0,
+                taskName,
+                deserializeBinary,
+                expiryPlc,
+                skipVals,
+                skipStore,
+                canRemap,
+                needVer);
+        }
     }
 
     /** {@inheritDoc} */
-    @Override public V getAndPut(K key, V val, @Nullable CacheEntryPredicate filter) throws IgniteCheckedException {
-        return getAndPutAsync0(key, val, filter).get();
+    @Override protected V getAndPut0(K key, V val, @Nullable CacheEntryPredicate filter) throws IgniteCheckedException {
+        return (V)update0(
+            key,
+            val,
+            null,
+            null,
+            true,
+            filter,
+            true,
+            false).get();
     }
 
     /** {@inheritDoc} */
-    @Override public boolean put(K key, V val, CacheEntryPredicate filter) throws IgniteCheckedException {
-        return putAsync(key, val, filter).get();
+    @Override protected boolean put0(K key, V val, CacheEntryPredicate filter) throws IgniteCheckedException {
+        Boolean res = (Boolean)update0(
+            key,
+            val,
+            null,
+            null,
+            false,
+            filter,
+            true,
+            false).get();
+
+        assert res != null;
+
+        return res;
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public IgniteInternalFuture<V> getAndPutAsync0(K key, V val, @Nullable CacheEntryPredicate filter) {
-        A.notNull(key, "key", val, "val");
-
-        return updateAsync0(
+        return update0(
             key,
             val,
             null,
             null,
             true,
             filter,
+            true,
             true);
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public IgniteInternalFuture<Boolean> putAsync0(K key, V val, @Nullable CacheEntryPredicate filter) {
-        return updateAsync0(
+        return update0(
             key,
             val,
             null,
             null,
             false,
             filter,
+            true,
             true);
     }
 
@@ -627,84 +718,34 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @Override public V tryGetAndPut(K key, V val) throws IgniteCheckedException {
         A.notNull(key, "key", val, "val");
 
-        return (V)updateAsync0(
+        return (V) update0(
             key,
             val,
             null,
             null,
             true,
             null,
+            false,
             false).get();
     }
 
     /** {@inheritDoc} */
-    @Override public V getAndPutIfAbsent(K key, V val) throws IgniteCheckedException {
-        return getAndPutIfAbsentAsync(key, val).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<V> getAndPutIfAbsentAsync(K key, V val) {
-        A.notNull(key, "key", val, "val");
-
-        return getAndPutAsync(key, val, ctx.noVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean putIfAbsent(K key, V val) throws IgniteCheckedException {
-        return putIfAbsentAsync(key, val).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> putIfAbsentAsync(K key, V val) {
-        A.notNull(key, "key", val, "val");
-
-        return putAsync(key, val, ctx.noVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public V getAndReplace(K key, V val) throws IgniteCheckedException {
-        return getAndReplaceAsync(key, val).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<V> getAndReplaceAsync(K key, V val) {
-        A.notNull(key, "key", val, "val");
-
-        return getAndPutAsync(key, val, ctx.hasVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean replace(K key, V val) throws IgniteCheckedException {
-        return replaceAsync(key, val).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> replaceAsync(K key, V val) {
-        A.notNull(key, "key", val, "val");
-
-        return putAsync(key, val, ctx.hasVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean replace(K key, V oldVal, V newVal) throws IgniteCheckedException {
-        return replaceAsync(key, oldVal, newVal).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
-        A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal");
-
-        return putAsync(key, newVal, ctx.equalsVal(oldVal));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void putAll(Map<? extends K, ? extends V> m) throws IgniteCheckedException {
-        putAllAsync(m).get();
+    @Override protected void putAll0(Map<? extends K, ? extends V> m) throws IgniteCheckedException {
+        updateAll0(m,
+            null,
+            null,
+            null,
+            null,
+            false,
+            false,
+            true,
+            UPDATE,
+            false).get();
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> putAllAsync(Map<? extends K, ? extends V> m) {
-        return updateAllAsync0(m,
+    @Override public IgniteInternalFuture<?> putAllAsync0(Map<? extends K, ? extends V> m) {
+        return updateAll0(m,
             null,
             null,
             null,
@@ -712,7 +753,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             false,
             false,
             true,
-            UPDATE).chain(RET2NULL);
+            UPDATE,
+            true).chain(RET2NULL);
     }
 
     /** {@inheritDoc} */
@@ -725,7 +767,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @Override public IgniteInternalFuture<?> putAllConflictAsync(Map<KeyCacheObject, GridCacheDrInfo> conflictMap) {
         ctx.dr().onReceiveCacheEntriesReceived(conflictMap.size());
 
-        return updateAllAsync0(null,
+        return updateAll0(null,
             null,
             null,
             conflictMap,
@@ -733,57 +775,40 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             false,
             false,
             true,
-            UPDATE);
+            UPDATE,
+            true);
     }
 
     /** {@inheritDoc} */
-    @Override public V getAndRemove(K key) throws IgniteCheckedException {
-        return getAndRemoveAsync(key).get();
+    @Override public V getAndRemove0(K key) throws IgniteCheckedException {
+        return (V)remove0(key, true, null, false).get();
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteInternalFuture<V> getAndRemoveAsync(K key) {
-        A.notNull(key, "key");
-
-        return removeAsync0(key, true, null);
+    @Override public IgniteInternalFuture<V> getAndRemoveAsync0(K key) {
+        return remove0(key, true, null, true);
     }
 
     /** {@inheritDoc} */
-    @Override public void removeAll(Collection<? extends K> keys) throws IgniteCheckedException {
-        removeAllAsync(keys).get();
+    @Override protected void removeAll0(Collection<? extends K> keys) throws IgniteCheckedException {
+        removeAllAsync0(keys, null, false, false, false).get();
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> removeAllAsync(Collection<? extends K> keys) {
-        A.notNull(keys, "keys");
-
-        return removeAllAsync0(keys, null, false, false).chain(RET2NULL);
+    @Override public IgniteInternalFuture<Object> removeAllAsync0(Collection<? extends K> keys) {
+        return removeAllAsync0(keys, null, false, false, true).chain(RET2NULL);
     }
 
     /** {@inheritDoc} */
-    @Override public boolean remove(K key) throws IgniteCheckedException {
-        return removeAsync(key, (CacheEntryPredicate)null).get();
+    @Override protected boolean remove0(K key, CacheEntryPredicate filter) throws IgniteCheckedException {
+        return (Boolean)remove0(key, false, filter, false).get();
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteInternalFuture<Boolean> removeAsync(K key, @Nullable CacheEntryPredicate filter) {
-        A.notNull(key, "key");
-
-        return removeAsync0(key, false, filter);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean remove(K key, V val) throws IgniteCheckedException {
-        return removeAsync(key, val).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> removeAsync(K key, V val) {
-        A.notNull(key, "key", val, "val");
-
-        return removeAsync(key, ctx.equalsVal(val));
+    @Override public IgniteInternalFuture<Boolean> removeAsync0(K key, @Nullable CacheEntryPredicate filter) {
+        return remove0(key, false, filter, true);
     }
 
     /** {@inheritDoc} */
@@ -796,7 +821,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @Override public IgniteInternalFuture<?> removeAllConflictAsync(Map<KeyCacheObject, GridCacheVersion> conflictMap) {
         ctx.dr().onReceiveCacheEntriesReceived(conflictMap.size());
 
-        return removeAllAsync0(null, conflictMap, false, false);
+        return removeAllAsync0(null, conflictMap, false, false, true);
     }
 
     /**
@@ -811,10 +836,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @return Future.
      */
     @SuppressWarnings("unchecked")
-    protected <T> IgniteInternalFuture<T> asyncOp(final CO<IgniteInternalFuture<T>> op) {
-        if (!asyncToggled)
-            return op.apply();
-
+    private <T> IgniteInternalFuture<T> asyncOp(final CO<IgniteInternalFuture<T>> op) {
         IgniteInternalFuture<T> fail = asyncOpAcquire();
 
         if (fail != null)
@@ -871,7 +893,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     /** {@inheritDoc} */
     @Override public <T> EntryProcessorResult<T> invoke(K key, EntryProcessor<K, V, T> entryProcessor, Object... args)
         throws IgniteCheckedException {
-        IgniteInternalFuture<EntryProcessorResult<T>> invokeFut = invokeAsync(key, entryProcessor, args);
+        IgniteInternalFuture<EntryProcessorResult<T>> invokeFut = invoke0(false, key, entryProcessor, args);
 
         EntryProcessorResult<T> res = invokeFut.get();
 
@@ -881,16 +903,30 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     /** {@inheritDoc} */
     @Override public <T> Map<K, EntryProcessorResult<T>> invokeAll(Set<? extends K> keys,
         EntryProcessor<K, V, T> entryProcessor,
-        Object... args)
-        throws IgniteCheckedException {
-        return invokeAllAsync(keys, entryProcessor, args).get();
+        Object... args) throws IgniteCheckedException
+    {
+        return invokeAll0(false, keys, entryProcessor, args).get();
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
     @Override public <T> IgniteInternalFuture<EntryProcessorResult<T>> invokeAsync(K key,
         EntryProcessor<K, V, T> entryProcessor,
         Object... args) {
+        return invoke0(true, key, entryProcessor, args);
+    }
+
+    /**
+     * @param async Async operation flag.
+     * @param key Key.
+     * @param entryProcessor Entry processor.
+     * @param args Entry processor arguments.
+     * @return Future.
+     */
+    private <T> IgniteInternalFuture<EntryProcessorResult<T>> invoke0(
+        boolean async,
+        K key,
+        EntryProcessor<K, V, T> entryProcessor,
+        Object... args) {
         A.notNull(key, "key", entryProcessor, "entryProcessor");
 
         if (keyCheck)
@@ -900,14 +936,15 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
         final boolean keepBinary = opCtx != null && opCtx.isKeepBinary();
 
-        IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> fut = updateAsync0(
+        IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> fut = update0(
             key,
             null,
             entryProcessor,
             args,
             false,
             null,
-            true);
+            true,
+            async);
 
         return fut.chain(new CX1<IgniteInternalFuture<Map<K, EntryProcessorResult<T>>>, EntryProcessorResult<T>>() {
             @Override public EntryProcessorResult<T> applyx(IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> fut)
@@ -940,6 +977,21 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @Override public <T> IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(Set<? extends K> keys,
         final EntryProcessor<K, V, T> entryProcessor,
         Object... args) {
+        return invokeAll0(true, keys, entryProcessor, args);
+    }
+
+    /**
+     * @param async Async operation flag.
+     * @param keys Keys.
+     * @param entryProcessor Entry processor.
+     * @param args Entry processor arguments.
+     * @return Future.
+     */
+    private <T> IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> invokeAll0(
+        boolean async,
+        Set<? extends K> keys,
+        final EntryProcessor<K, V, T> entryProcessor,
+        Object... args) {
         A.notNull(keys, "keys", entryProcessor, "entryProcessor");
 
         if (keyCheck)
@@ -955,7 +1007,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
         final boolean keepBinary = opCtx != null && opCtx.isKeepBinary();
 
-        IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> resFut = updateAllAsync0(null,
+        IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> resFut = updateAll0(null,
             invokeMap,
             args,
             null,
@@ -963,7 +1015,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             false,
             false,
             true,
-            TRANSFORM);
+            TRANSFORM,
+            async);
 
         return resFut.chain(
             new CX1<IgniteInternalFuture<Map<K, EntryProcessorResult<T>>>, Map<K, EntryProcessorResult<T>>>() {
@@ -981,7 +1034,21 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @Override public <T> Map<K, EntryProcessorResult<T>> invokeAll(
         Map<? extends K, ? extends EntryProcessor<K, V, T>> map,
         Object... args) throws IgniteCheckedException {
-        return invokeAllAsync(map, args).get();
+        A.notNull(map, "map");
+
+        if (keyCheck)
+            validateCacheKeys(map.keySet());
+
+        return (Map<K, EntryProcessorResult<T>>)updateAll0(null,
+            map,
+            args,
+            null,
+            null,
+            false,
+            false,
+            true,
+            TRANSFORM,
+            false).get();
     }
 
     /** {@inheritDoc} */
@@ -994,7 +1061,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         if (keyCheck)
             validateCacheKeys(map.keySet());
 
-        return updateAllAsync0(null,
+        return updateAll0(null,
             map,
             args,
             null,
@@ -1002,7 +1069,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             false,
             false,
             true,
-            TRANSFORM);
+            TRANSFORM,
+            true);
     }
 
     /**
@@ -1016,10 +1084,11 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @param retval Return value required flag.
      * @param rawRetval Return {@code GridCacheReturn} instance.
      * @param waitTopFut Whether to wait for topology future.
+     * @param async Async operation flag.
      * @return Completion future.
      */
     @SuppressWarnings("ConstantConditions")
-    private IgniteInternalFuture updateAllAsync0(
+    private IgniteInternalFuture updateAll0(
         @Nullable Map<? extends K, ? extends V> map,
         @Nullable Map<? extends K, ? extends EntryProcessor> invokeMap,
         @Nullable Object[] invokeArgs,
@@ -1028,7 +1097,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         final boolean retval,
         final boolean rawRetval,
         final boolean waitTopFut,
-        final GridCacheOperation op
+        final GridCacheOperation op,
+        boolean async
     ) {
         assert ctx.updatesAllowed();
 
@@ -1105,13 +1175,20 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             opCtx != null && opCtx.noRetries() ? 1 : MAX_RETRIES,
             waitTopFut);
 
-        return asyncOp(new CO<IgniteInternalFuture<Object>>() {
-            @Override public IgniteInternalFuture<Object> apply() {
-                updateFut.map();
+        if (async) {
+            return asyncOp(new CO<IgniteInternalFuture<Object>>() {
+                @Override public IgniteInternalFuture<Object> apply() {
+                    updateFut.map();
 
-                return updateFut;
-            }
-        });
+                    return updateFut;
+                }
+            });
+        }
+        else {
+            updateFut.map();
+
+            return updateFut;
+        }
     }
 
     /**
@@ -1124,16 +1201,18 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @param retval Return value flag.
      * @param filter Filter.
      * @param waitTopFut Whether to wait for topology future.
+     * @param async Async operation flag.
      * @return Future.
      */
-    private IgniteInternalFuture updateAsync0(
+    private IgniteInternalFuture update0(
         K key,
         @Nullable V val,
         @Nullable EntryProcessor proc,
         @Nullable Object[] invokeArgs,
         final boolean retval,
         @Nullable final CacheEntryPredicate filter,
-        final boolean waitTopFut
+        final boolean waitTopFut,
+        boolean async
     ) {
         assert val == null || proc == null;
 
@@ -1146,13 +1225,20 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         final GridNearAtomicAbstractUpdateFuture updateFut =
             createSingleUpdateFuture(key, val, proc, invokeArgs, retval, filter, waitTopFut);
 
-        return asyncOp(new CO<IgniteInternalFuture<Object>>() {
-            @Override public IgniteInternalFuture<Object> apply() {
-                updateFut.map();
+        if (async) {
+            return asyncOp(new CO<IgniteInternalFuture<Object>>() {
+                @Override public IgniteInternalFuture<Object> apply() {
+                    updateFut.map();
 
-                return updateFut;
-            }
-        });
+                    return updateFut;
+                }
+            });
+        }
+        else {
+            updateFut.map();
+
+            return updateFut;
+        }
     }
 
     /**
@@ -1161,33 +1247,38 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @param key Key.
      * @param retval Whether to return
      * @param filter Filter.
+     * @param async Async operation flag.
      * @return Future.
      */
-    private IgniteInternalFuture removeAsync0(K key, final boolean retval,
-        @Nullable CacheEntryPredicate filter) {
-        final boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
+    private IgniteInternalFuture remove0(K key, final boolean retval,
+        @Nullable CacheEntryPredicate filter,
+        boolean async) {
         assert ctx.updatesAllowed();
 
-        validateCacheKey(key);
-
         ctx.checkSecurity(SecurityPermission.CACHE_REMOVE);
 
-        final GridNearAtomicAbstractUpdateFuture updateFut =
-            createSingleUpdateFuture(key, null, null, null, retval, filter, true);
+        final GridNearAtomicAbstractUpdateFuture updateFut = createSingleUpdateFuture(key,
+            null,
+            null,
+            null,
+            retval,
+            filter,
+            true);
 
-        if (statsEnabled)
-            updateFut.listen(new UpdateRemoveTimeStatClosure<>(metrics0(), start));
+        if (async) {
+            return asyncOp(new CO<IgniteInternalFuture<Object>>() {
+                @Override public IgniteInternalFuture<Object> apply() {
+                    updateFut.map();
 
-        return asyncOp(new CO<IgniteInternalFuture<Object>>() {
-            @Override public IgniteInternalFuture<Object> apply() {
-                updateFut.map();
+                    return updateFut;
+                }
+            });
+        }
+        else {
+            updateFut.map();
 
-                return updateFut;
-            }
-        });
+            return updateFut;
+        }
     }
 
     /**
@@ -1326,14 +1417,11 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         @Nullable Collection<? extends K> keys,
         @Nullable Map<KeyCacheObject, GridCacheVersion> conflictMap,
         final boolean retval,
-        boolean rawRetval
+        boolean rawRetval,
+        boolean async
     ) {
         assert ctx.updatesAllowed();
 
-        final boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
         assert keys != null || conflictMap != null;
 
         if (keyCheck)
@@ -1380,16 +1468,20 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             opCtx != null && opCtx.noRetries() ? 1 : MAX_RETRIES,
             true);
 
-        if (statsEnabled)
-            updateFut.listen(new UpdateRemoveTimeStatClosure<>(metrics0(), start));
+        if (async) {
+            return asyncOp(new CO<IgniteInternalFuture<Object>>() {
+                @Override public IgniteInternalFuture<Object> apply() {
+                    updateFut.map();
 
-        return asyncOp(new CO<IgniteInternalFuture<Object>>() {
-            @Override public IgniteInternalFuture<Object> apply() {
-                updateFut.map();
+                    return updateFut;
+                }
+            });
+        }
+        else {
+            updateFut.map();
 
-                return updateFut;
-            }
-        });
+            return updateFut;
+        }
     }
 
     /**
@@ -3248,7 +3340,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @param ver Version to ack.
      */
     private void sendDeferredUpdateResponse(UUID nodeId, GridCacheVersion ver) {
-        deferredUpdateMessageSender.sendDeferredAckMessage(nodeId, ver);
+        deferredUpdateMsgSnd.sendDeferredAckMessage(nodeId, ver);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/c530d47b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
index 176a90f..9cf400d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
@@ -327,19 +327,6 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
             needVer);
     }
 
-    /** {@inheritDoc} */
-    @Override protected GridCacheEntryEx entryExSafe(
-        KeyCacheObject key,
-        AffinityTopologyVersion topVer
-    ) {
-        try {
-            return ctx.affinity().localNode(key, topVer) ? entryEx(key) : null;
-        }
-        catch (GridDhtInvalidPartitionException ignored) {
-            return null;
-        }
-    }
-
     /**
      * @param keys Keys to load.
      * @param readThrough Read through flag.


[14/16] ignite git commit: IGNITE-1443: Implemented ContinuousQuery for C++

Posted by vo...@apache.org.
IGNITE-1443: Implemented ContinuousQuery for C++

This closes #1343


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

Branch: refs/heads/ignite-2.0
Commit: 598b464f81d2e74dc0df62011e08f3a76a674db6
Parents: 700529a
Author: Igor Sapego <is...@gridgain.com>
Authored: Tue Dec 20 19:38:41 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Dec 20 19:38:41 2016 +0300

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 .../include/ignite/binary/binary_raw_reader.h   |  21 +
 .../ignite/impl/binary/binary_reader_impl.h     |   9 +-
 .../src/impl/binary/binary_reader_impl.cpp      |  17 +
 .../platforms/cpp/common/include/Makefile.am    |   5 +-
 .../common/include/ignite/common/concurrent.h   |  90 ++-
 .../include/ignite/common/reference_impl.h      | 286 +++++++++
 .../cpp/common/include/ignite/reference.h       | 564 +++++++++++++++++
 .../cpp/common/project/vs/common.vcxproj        |   2 +
 .../common/project/vs/common.vcxproj.filters    |   6 +
 modules/platforms/cpp/core-test/Makefile.am     |   7 +-
 .../core-test/config/cache-query-continuous.xml |  87 +++
 .../cpp/core-test/project/vs/core-test.vcxproj  |  10 +-
 .../project/vs/core-test.vcxproj.filters        |   6 +
 .../cpp/core-test/src/cache_query_test.cpp      |  76 +--
 .../cpp/core-test/src/continuous_query_test.cpp | 611 +++++++++++++++++++
 .../cpp/core-test/src/handle_registry_test.cpp  |  18 +-
 .../cpp/core-test/src/reference_test.cpp        | 412 +++++++++++++
 modules/platforms/cpp/core/Makefile.am          |   1 +
 .../cpp/core/include/ignite/cache/cache.h       | 104 +++-
 .../cpp/core/include/ignite/cache/cache_entry.h |  40 +-
 .../ignite/cache/event/cache_entry_event.h      | 139 +++++
 .../cache/event/cache_entry_event_listener.h    |  71 +++
 .../cache/query/continuous/continuous_query.h   | 239 ++++++++
 .../query/continuous/continuous_query_handle.h  | 133 ++++
 .../core/include/ignite/impl/cache/cache_impl.h | 116 +++-
 .../continuous/continuous_query_handle_impl.h   | 101 +++
 .../query/continuous/continuous_query_impl.h    | 351 +++++++++++
 .../core/include/ignite/impl/handle_registry.h  |  62 +-
 .../include/ignite/impl/ignite_environment.h    |  34 +-
 modules/platforms/cpp/core/namespaces.dox       |  74 ++-
 .../platforms/cpp/core/project/vs/core.vcxproj  |   7 +
 .../cpp/core/project/vs/core.vcxproj.filters    |  30 +
 .../cpp/core/src/impl/cache/cache_impl.cpp      |  31 +
 .../continuous/continuous_query_handle_impl.cpp |  96 +++
 .../cpp/core/src/impl/handle_registry.cpp       | 102 ++--
 .../cpp/core/src/impl/ignite_environment.cpp    |  98 ++-
 modules/platforms/cpp/examples/Makefile.am      |   1 +
 modules/platforms/cpp/examples/configure.ac     |   1 +
 .../continuous-query-example/Makefile.am        |  58 ++
 .../config/continuous-query-example.xml         |  52 ++
 .../project/vs/continuous-query-example.vcxproj | 110 ++++
 .../vs/continuous-query-example.vcxproj.filters |  35 ++
 .../src/continuous_query_example.cpp            | 142 +++++
 .../examples/include/ignite/examples/person.h   |   2 +-
 .../cpp/examples/project/vs/ignite-examples.sln |   6 +
 .../platforms/cpp/jni/include/ignite/jni/java.h |   2 +-
 modules/platforms/cpp/jni/src/java.cpp          |   4 +-
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |   4 +-
 modules/platforms/cpp/project/vs/ignite.slnrel  |   3 +
 .../platforms/cpp/project/vs/ignite_x86.slnrel  |   3 +
 51 files changed, 4216 insertions(+), 264 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index b534eb7..db6128e 100644
--- a/.gitignore
+++ b/.gitignore
@@ -51,6 +51,7 @@ git-patch-prop-local.sh
 *.opensdf
 *.db
 *.opendb
+.vs
 ipch/
 [Oo]bj/
 [Bb]in

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h b/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h
index 72aab55..d15848b 100644
--- a/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h
+++ b/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h
@@ -398,6 +398,27 @@ namespace ignite
             {
                 return impl->ReadObject<T>();
             }
+
+            /**
+             * Try read object.
+             * Reads value, stores it to res and returns true if the value is
+             * not null. Otherwise just returns false.
+             *
+             * @param res Read value is placed here if non-null.
+             * @return True if the non-null value has been read and false
+             *     otherwise.
+             */
+            template<typename T>
+            bool TryReadObject(T& res)
+            {
+                if (impl->SkipIfNull())
+                    return false;
+
+                res = impl->ReadObject<T>();
+
+                return true;
+            }
+
         private:
             /** Implementation delegate. */
             ignite::impl::binary::BinaryReaderImpl* impl;  

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h
index 8c4b464..cd32203 100644
--- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h
+++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h
@@ -723,6 +723,13 @@ namespace ignite
                 bool HasNextElement(int32_t id) const;
 
                 /**
+                 * Skip next value if it is the null.
+                 *
+                 * @return True if the null value has been detected and skipped.
+                 */
+                bool SkipIfNull();
+
+                /**
                  * Read element.
                  *
                  * @param id Session ID.
@@ -763,7 +770,7 @@ namespace ignite
                     *key = ReadTopObject<K>();
                     *val = ReadTopObject<V>();
                 }
-                
+
                 /**
                  * Read object.
                  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp b/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp
index fb75ba5..c128df6 100644
--- a/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp
+++ b/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp
@@ -663,6 +663,23 @@ namespace ignite
                 return elemId == id && elemRead < elemCnt;
             }
 
+            bool BinaryReaderImpl::SkipIfNull()
+            {
+                CheckRawMode(true);
+                CheckSingleMode(true);
+
+                InteropStreamPositionGuard<InteropInputStream> positionGuard(*stream);
+
+                int8_t hdr = stream->ReadInt8();
+
+                if (hdr != IGNITE_HDR_NULL)
+                    return false;
+
+                positionGuard.Release();
+
+                return true;
+            }
+
             void BinaryReaderImpl::SetRawMode()
             {
                 CheckRawMode(false);

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/common/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/Makefile.am b/modules/platforms/cpp/common/include/Makefile.am
index 2e53608..a5073f6 100644
--- a/modules/platforms/cpp/common/include/Makefile.am
+++ b/modules/platforms/cpp/common/include/Makefile.am
@@ -23,14 +23,15 @@ nobase_include_HEADERS = \
 	ignite/common/concurrent.h \
 	ignite/common/decimal.h \
 	ignite/common/default_allocator.h \
+	ignite/common/reference_impl.h \
 	ignite/common/dynamic_size_array.h \
 	ignite/common/fixed_size_array.h \
 	ignite/common/utils.h \
 	ignite/date.h \
 	ignite/guid.h \
 	ignite/ignite_error.h \
-	ignite/timestamp.h
-	ignite/timestamp.h
+	ignite/timestamp.h \
+	ignite/reference.h
 
 uninstall-hook:
 	if [ -d ${includedir}/ignite ]; then find ${includedir}/ignite -type d -empty -delete; fi

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/common/include/ignite/common/concurrent.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/common/concurrent.h b/modules/platforms/cpp/common/include/ignite/common/concurrent.h
index 35c2209..ff0e54a 100644
--- a/modules/platforms/cpp/common/include/ignite/common/concurrent.h
+++ b/modules/platforms/cpp/common/include/ignite/common/concurrent.h
@@ -19,6 +19,7 @@
 #define _IGNITE_COMMON_CONCURRENT
 
 #include <cassert>
+#include <utility>
 
 #include "ignite/common/concurrent_os.h"
 
@@ -121,10 +122,15 @@ namespace ignite
             public:
                 friend class EnableSharedFromThis<T>;
 
+                template<typename T2>
+                friend class SharedPointer;
+
                 /**
                  * Constructor.
                  */
-                SharedPointer() : impl(0)
+                SharedPointer() :
+                    ptr(0),
+                    impl(0)
                 {
                     // No-op.
                 }
@@ -133,17 +139,17 @@ namespace ignite
                  * Constructor.
                  *
                  * @param ptr Raw pointer.
+                 * @param deleter Delete function.
                  */
-                explicit SharedPointer(T* ptr)
+                SharedPointer(T* ptr, void(*deleter)(T*) = &SharedPointerDefaultDeleter<T>) :
+                    ptr(ptr),
+                    impl(0)
                 {
                     if (ptr)
                     {
-                        void(*deleter)(T*) = (void(*)(T*)) &SharedPointerDefaultDeleter<T>;
                         impl = new SharedPointerImpl(ptr, reinterpret_cast<SharedPointerImpl::DeleterType>(deleter));
                         ImplEnableShared(ptr, impl);
                     }
-                    else
-                        impl = 0;
                 }
 
                 /**
@@ -152,15 +158,16 @@ namespace ignite
                  * @param ptr Raw pointer.
                  * @param deleter Delete function.
                  */
-                SharedPointer(T* ptr, void(*deleter)(T*))
+                template<typename T2>
+                SharedPointer(T2* ptr, void(*deleter)(T2*) = &SharedPointerDefaultDeleter<T2>) :
+                    ptr(ptr),
+                    impl(0)
                 {
                     if (ptr)
                     {
                         impl = new SharedPointerImpl(ptr, reinterpret_cast<SharedPointerImpl::DeleterType>(deleter));
                         ImplEnableShared(ptr, impl);
                     }
-                    else
-                        impl = 0;
                 }
 
                 /**
@@ -169,6 +176,21 @@ namespace ignite
                  * @param other Instance to copy.
                  */
                 SharedPointer(const SharedPointer& other) :
+                    ptr(other.ptr),
+                    impl(other.impl)
+                {
+                    if (impl)
+                        impl->Increment();
+                }
+
+                /**
+                 * Copy constructor.
+                 *
+                 * @param other Instance to copy.
+                 */
+                template<typename T2>
+                SharedPointer(const SharedPointer<T2>& other) :
+                    ptr(other.ptr),
                     impl(other.impl)
                 {
                     if (impl)
@@ -186,26 +208,43 @@ namespace ignite
                     {
                         SharedPointer tmp(other);
 
-                        std::swap(impl, tmp.impl);
+                        Swap(tmp);
                     }
 
                     return *this;
                 }
 
                 /**
+                 * Assignment operator.
+                 *
+                 * @param other Other instance.
+                 */
+                template<typename T2>
+                SharedPointer& operator=(const SharedPointer<T2>& other)
+                {
+                    SharedPointer<T> tmp(other);
+
+                    Swap(tmp);
+
+                    return *this;
+                }
+
+                /**
                  * Destructor.
                  */
                 ~SharedPointer()
                 {
                     if (impl && impl->Decrement())
                     {
-                        T* ptr = Get();
+                        void* ptr0 = impl->Pointer();
 
-                        void(*deleter)(T*) = reinterpret_cast<void(*)(T*)>(impl->Deleter());
+                        void(*deleter)(void*) = impl->Deleter();
 
-                        deleter(ptr);
+                        deleter(ptr0);
 
                         delete impl;
+
+                        ptr = 0;
                     }
                 }
 
@@ -216,7 +255,7 @@ namespace ignite
                  */
                 T* Get()
                 {
-                    return impl ? static_cast<T*>(impl->Pointer()) : 0;
+                    return ptr;
                 }
 
                 /**
@@ -226,7 +265,7 @@ namespace ignite
                  */
                 const T* Get() const
                 {
-                    return impl ? static_cast<T*>(impl->Pointer()) : 0;
+                    return ptr;
                 }
 
                 /**
@@ -245,7 +284,30 @@ namespace ignite
                     return impl != 0;
                 }
 
+                /**
+                 * Swap pointer content with another instance.
+                 *
+                 * @param other Other instance.
+                 */
+                void Swap(SharedPointer& other)
+                {
+                    if (this != &other)
+                    {
+                        T* ptrTmp = ptr;
+                        SharedPointerImpl* implTmp = impl;
+
+                        ptr = other.ptr;
+                        impl = other.impl;
+
+                        other.ptr = ptrTmp;
+                        other.impl = implTmp;
+                    }
+                }
+
             private:
+                /* Pointer. */
+                T* ptr;
+
                 /** Implementation. */
                 SharedPointerImpl* impl;
             };

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/common/include/ignite/common/reference_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/common/reference_impl.h b/modules/platforms/cpp/common/include/ignite/common/reference_impl.h
new file mode 100644
index 0000000..e38da8c
--- /dev/null
+++ b/modules/platforms/cpp/common/include/ignite/common/reference_impl.h
@@ -0,0 +1,286 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::ReferenceImplBase class and its implementations.
+ */
+
+#ifndef _IGNITE_COMMON_REFERENCE_IMPL
+#define _IGNITE_COMMON_REFERENCE_IMPL
+
+#include <utility>
+
+#include <ignite/common/common.h>
+
+namespace ignite
+{
+    namespace common
+    {
+        // Any number is good as long as it is not null.
+        enum { POINTER_CAST_MAGIC_NUMBER = 80000 };
+
+        /**
+         * Interface for constant Reference implementation class template.
+         */
+        class ConstReferenceImplBase
+        {
+        public:
+            /**
+             * Destructor.
+             */
+            virtual ~ConstReferenceImplBase()
+            {
+                // No-op.
+            }
+
+            /**
+             * Get the constant pointer.
+             *
+             * @return Constant pointer to underlying value.
+             */
+            virtual const void* Get() const = 0;
+        };
+
+        /**
+         * Interface for Reference implementation class template.
+         */
+        class ReferenceImplBase : public ConstReferenceImplBase
+        {
+        public:
+            /**
+             * Destructor.
+             */
+            virtual ~ReferenceImplBase()
+            {
+                // No-op.
+            }
+
+            virtual const void* Get() const = 0;
+
+            /**
+             * Get the pointer.
+             *
+             * @return Pointer to underlying value.
+             */
+            virtual void* Get() = 0;
+        };
+
+        /**
+         * Reference class implementation for smart pointers.
+         *
+         * Note, this class does not implement any smart pointer functionality
+         * itself, instead it wraps one of the existing wide-spread smart
+         * pointer implementations and provides unified interface for them.
+         */
+        template<typename P>
+        class ReferenceSmartPointer : public ReferenceImplBase
+        {
+        public:
+            /**
+             * Destructor.
+             */
+            virtual ~ReferenceSmartPointer()
+            {
+                // No-op.
+            }
+
+            /**
+             * Default constructor.
+             */
+            ReferenceSmartPointer() :
+                ptr()
+            {
+                // No-op.
+            }
+
+            virtual const void* Get() const
+            {
+                return reinterpret_cast<const void*>(&(*ptr));
+            }
+
+            virtual void* Get()
+            {
+                return reinterpret_cast<void*>(&(*ptr));
+            }
+
+            /**
+             * Swap underlying smart pointer.
+             *
+             * @param other Another instance.
+             */
+            void Swap(P& other)
+            {
+                using std::swap;
+
+                swap(ptr, other);
+            }
+
+        private:
+            /** Underlying pointer. */
+            P ptr;
+        };
+
+        /**
+         * Reference implementation for the owning raw pointer.
+         */
+        template<typename T>
+        class ReferenceOwningRawPointer : public ReferenceImplBase
+        {
+        public:
+            /**
+             * Destructor.
+             */
+            virtual ~ReferenceOwningRawPointer()
+            {
+                delete ptr;
+            }
+
+            /**
+             * Default constructor.
+             */
+            ReferenceOwningRawPointer() :
+                ptr(0)
+            {
+                // No-op.
+            }
+
+            /**
+             * Pointer constructor.
+             *
+             * @param ptr Pointer to take ownership over.
+             */
+            ReferenceOwningRawPointer(T* ptr) :
+                ptr(ptr)
+            {
+                // No-op.
+            }
+
+            virtual const void* Get() const
+            {
+                return reinterpret_cast<const void*>(ptr);
+            }
+
+            virtual void* Get()
+            {
+                return reinterpret_cast<void*>(ptr);
+            }
+
+        private:
+            /** Underlying pointer. */
+            T* ptr;
+        };
+
+        /**
+         * Reference implementation for the raw pointer.
+         */
+        template<typename T>
+        class ReferenceNonOwningRawPointer : public ReferenceImplBase
+        {
+        public:
+            /**
+             * Destructor.
+             */
+            virtual ~ReferenceNonOwningRawPointer()
+            {
+                // No-op.
+            }
+
+            /**
+             * Default constructor.
+             */
+            ReferenceNonOwningRawPointer() :
+                ptr(0)
+            {
+                // No-op.
+            }
+
+            /**
+             * Pointer constructor.
+             *
+             * @param ptr Pointer.
+             */
+            ReferenceNonOwningRawPointer(T* ptr) :
+                ptr(ptr)
+            {
+                // No-op.
+            }
+
+            virtual const void* Get() const
+            {
+                return reinterpret_cast<const void*>(ptr);
+            }
+
+            virtual void* Get()
+            {
+                return reinterpret_cast<void*>(ptr);
+            }
+
+        private:
+            /** Underlying pointer. */
+            T* ptr;
+        };
+
+        /**
+         * Constant reference implementation for the raw pointer.
+         */
+        template<typename T>
+        class ConstReferenceNonOwningRawPointer : public ConstReferenceImplBase
+        {
+        public:
+            /**
+             * Destructor.
+             */
+            virtual ~ConstReferenceNonOwningRawPointer()
+            {
+                // No-op.
+            }
+
+            /**
+             * Default constructor.
+             */
+            ConstReferenceNonOwningRawPointer() :
+                ptr(0)
+            {
+                // No-op.
+            }
+
+            /**
+             * Pointer constructor.
+             *
+             * @param ptr Pointer.
+             */
+            ConstReferenceNonOwningRawPointer(const T* ptr) :
+                ptr(ptr)
+            {
+                // No-op.
+            }
+
+            virtual const void* Get() const
+            {
+                return reinterpret_cast<const void*>(ptr);
+            }
+
+        private:
+            /** Underlying pointer. */
+            const T* ptr;
+        };
+
+    }
+}
+
+#endif //_IGNITE_COMMON_REFERENCE_IMPL
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/common/include/ignite/reference.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/reference.h b/modules/platforms/cpp/common/include/ignite/reference.h
new file mode 100644
index 0000000..b026ad7
--- /dev/null
+++ b/modules/platforms/cpp/common/include/ignite/reference.h
@@ -0,0 +1,564 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::Reference class.
+ */
+
+#ifndef _IGNITE_COMMON_REFERENCE
+#define _IGNITE_COMMON_REFERENCE
+
+#include <cstddef>
+
+#include <ignite/common/common.h>
+#include <ignite/common/concurrent.h>
+#include <ignite/common/reference_impl.h>
+
+namespace ignite
+{
+    template<typename T>
+    class Reference;
+
+    /**
+     * Constant Reference class.
+     *
+     * Abstraction on any reference-type object, from simple raw pointers and
+     * references to standard library smart pointers. Provides only constant
+     * access to the underlying data.
+     *
+     * There are no requirements for the template type T.
+     */
+    template<typename T>
+    class ConstReference
+    {
+        template<typename>
+        friend class ConstReference;
+
+        template<typename>
+        friend class Reference;
+
+    public:
+        /**
+         * Default constructor.
+         */
+        ConstReference() :
+            ptr(),
+            offset(0)
+        {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param ptr ConstReference class implementation.
+         * @param offset Pointer offset.
+         */
+        explicit ConstReference(common::ConstReferenceImplBase* ptr, ptrdiff_t offset = 0) :
+            ptr(ptr),
+            offset(offset)
+        {
+            // No-op.
+        }
+
+        /**
+         * Copy constructor.
+         *
+         * @param other Another instance.
+         */
+        ConstReference(const ConstReference& other) :
+            ptr(other.ptr),
+            offset(other.offset)
+        {
+            // No-op.
+        }
+
+        /**
+         * Copy constructor.
+         *
+         * Constant reference of type T2 should be static-castable to constant
+         * reference of type T.
+         *
+         * @param other Another instance.
+         */
+        template<typename T2>
+        ConstReference(const ConstReference<T2>& other) :
+            ptr(other.ptr),
+            offset(other.offset)
+        {
+            T2* p0 = reinterpret_cast<T2*>(common::POINTER_CAST_MAGIC_NUMBER);
+            T* p1 = static_cast<T*>(p0);
+
+            ptrdiff_t diff = reinterpret_cast<ptrdiff_t>(p1) - reinterpret_cast<ptrdiff_t>(p0);
+            offset += diff;
+        }
+
+        /**
+         * Assignment operator.
+         *
+         * @param other Another instance.
+         */
+        ConstReference& operator=(const ConstReference& other)
+        {
+            ptr = other.ptr;
+            offset = other.offset;
+
+            return *this;
+        }
+        
+        /**
+         * Assignment operator.
+         *
+         * Constant reference of type T2 should be static-castable to constant
+         * reference of type T.
+         *
+         * @param other Another instance.
+         */
+        template<typename T2>
+        ConstReference& operator=(const ConstReference<T2>& other)
+        {
+            ptr = other.ptr;
+            offset = other.offset;
+
+            T2* p0 = reinterpret_cast<T2*>(common::POINTER_CAST_MAGIC_NUMBER);
+            T* p1 = static_cast<T*>(p0);
+
+            ptrdiff_t diff = reinterpret_cast<ptrdiff_t>(p1) - reinterpret_cast<ptrdiff_t>(p0);
+            offset += diff;
+
+            return *this;
+        }
+
+        /**
+         * Destructor.
+         */
+        ~ConstReference()
+        {
+            // No-op.
+        }
+
+        /**
+         * Dereference the pointer.
+         *
+         * If the pointer is null then this operation causes undefined
+         * behaviour.
+         *
+         * @return Constant reference to underlying value.
+         */
+        const T& Get() const
+        {
+            return *reinterpret_cast<const T*>(reinterpret_cast<ptrdiff_t>(ptr.Get()->Get()) + offset);
+        }
+
+        /**
+         * Check if the pointer is null.
+         *
+         * @return True if the value is null.
+         */
+        bool IsNull() const
+        {
+            const common::ConstReferenceImplBase* raw = ptr.Get();
+
+            return !raw || !raw->Get();
+        }
+
+    private:
+        /** Implementation. */
+        common::concurrent::SharedPointer<common::ConstReferenceImplBase> ptr;
+
+        /** Address offset. */
+        ptrdiff_t offset;
+    };
+
+    /**
+     * Reference class.
+     *
+     * Abstraction on any reference-type object, from simple raw pointers and
+     * references to standard library smart pointers.
+     *
+     * There are no requirements for the template type T.
+     */
+    template<typename T>
+    class Reference
+    {
+        template<typename>
+        friend class Reference;
+    public:
+        /**
+         * Default constructor.
+         */
+        Reference() :
+            ptr(),
+            offset(0)
+        {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param ptr Reference class implementation.
+         * @param offset Pointer offset.
+         */
+        explicit Reference(common::ReferenceImplBase* ptr, ptrdiff_t offset = 0) :
+            ptr(ptr),
+            offset(offset)
+        {
+            // No-op.
+        }
+
+        /**
+         * Copy constructor.
+         *
+         * @param other Another instance.
+         */
+        Reference(const Reference& other) :
+            ptr(other.ptr),
+            offset(other.offset)
+        {
+            // No-op.
+        }
+
+        /**
+         * Copy constructor.
+         * 
+         * Reference of type T2 should be static-castable to reference of type T.
+         *
+         * @param other Another instance.
+         */
+        template<typename T2>
+        Reference(const Reference<T2>& other) :
+            ptr(other.ptr),
+            offset(other.offset)
+        {
+            T2* p0 = reinterpret_cast<T2*>(common::POINTER_CAST_MAGIC_NUMBER);
+            T* p1 = static_cast<T*>(p0);
+
+            ptrdiff_t diff = reinterpret_cast<ptrdiff_t>(p1) - reinterpret_cast<ptrdiff_t>(p0);
+            offset += diff;
+        }
+
+        /**
+         * Assignment operator.
+         *
+         * @param other Another instance.
+         */
+        Reference& operator=(const Reference& other)
+        {
+            ptr = other.ptr;
+            offset = other.offset;
+
+            return *this;
+        }
+        
+        /**
+         * Assignment operator.
+         *
+         * Reference of type T2 should be static-castable to reference of type T.
+         *
+         * @param other Another instance.
+         */
+        template<typename T2>
+        Reference& operator=(const Reference<T2>& other)
+        {
+            ptr = other.ptr;
+            offset = other.offset;
+
+            T2* p0 = reinterpret_cast<T2*>(common::POINTER_CAST_MAGIC_NUMBER);
+            T* p1 = static_cast<T*>(p0);
+
+            ptrdiff_t diff = reinterpret_cast<ptrdiff_t>(p1) - reinterpret_cast<ptrdiff_t>(p0);
+            offset += diff;
+
+            return *this;
+        }
+
+        /**
+         * Destructor.
+         */
+        ~Reference()
+        {
+            // No-op.
+        }
+
+        /**
+         * Const cast operator.
+         *
+         * Reference of type T2 should be static-castable to reference of type T.
+         *
+         * Casts this instance to constant reference.
+         */
+        template<typename T2>
+        operator ConstReference<T2>()
+        {
+            ConstReference<T2> cr;
+
+            cr.ptr = ptr;
+            cr.offset = offset;
+
+            T2* p0 = reinterpret_cast<T2*>(common::POINTER_CAST_MAGIC_NUMBER);
+            const T* p1 = static_cast<T*>(p0);
+
+            ptrdiff_t diff = reinterpret_cast<ptrdiff_t>(p1) - reinterpret_cast<ptrdiff_t>(p0);
+            cr.offset -= diff;
+
+            return cr;
+        }
+
+        /**
+         * Dereference the pointer.
+         *
+         * If the pointer is null then this operation causes undefined
+         * behaviour.
+         *
+         * @return Constant reference to underlying value.
+         */
+        const T& Get() const
+        {
+            return *reinterpret_cast<const T*>(reinterpret_cast<ptrdiff_t>(ptr.Get()->Get()) + offset);
+        }
+
+        /**
+         * Dereference the pointer.
+         *
+         * If the pointer is null then this operation causes undefined
+         * behaviour.
+         *
+         * @return Reference to underlying value.
+         */
+        T& Get()
+        {
+            return *reinterpret_cast<T*>(reinterpret_cast<ptrdiff_t>(ptr.Get()->Get()) + offset);
+        }
+
+        /**
+         * Check if the pointer is null.
+         *
+         * @return True if the value is null.
+         */
+        bool IsNull() const
+        {
+            const common::ReferenceImplBase* raw = ptr.Get();
+
+            return !raw || !raw->Get();
+        }
+
+    private:
+        /** Implementation. */
+        common::concurrent::SharedPointer<common::ReferenceImplBase> ptr;
+
+        /** Address offset. */
+        ptrdiff_t offset;
+    };
+
+    /**
+     * Make ignite::Reference instance out of smart pointer.
+     *
+     * Template type 'T' should be a smart pointer and provide pointer semantics:
+     * - There should be defined type 'T::element_type', showing underlying type.
+     * - Type 'T' should be dereferencible (should have operators
+     *   T::element_type& operator*() and const T::element_type& operator*() const).
+     * - Operation std::swap should result in valid result if applied to two
+     *   instances of that type.
+     *
+     * @param ptr Pointer.
+     * @return Implementation defined value. User should not explicitly use the
+     *     returned value.
+     */
+    template<typename T>
+    Reference<typename T::element_type> MakeReferenceFromSmartPointer(T ptr)
+    {
+        common::ReferenceSmartPointer<T>* impl = new common::ReferenceSmartPointer<T>();
+
+        Reference<typename T::element_type> res(impl);
+
+        impl->Swap(ptr);
+
+        return res;
+    }
+
+    /**
+     * Make ignite::ConstReference instance out of smart pointer.
+     *
+     * Template type 'T' should be a smart pointer and provide pointer semantics:
+     * - There should be defined type 'T::element_type', showing underlying type.
+     * - Type 'T' should be dereferencible (should have operators
+     *   T::element_type& operator*() and const T::element_type& operator*() const).
+     * - Operation std::swap should result in valid result if applied to two
+     *   instances of that type.
+     *
+     * @param ptr Pointer.
+     * @return Implementation defined value. User should not explicitly use the
+     *     returned value.
+     */
+    template<typename T>
+    ConstReference<typename T::element_type> MakeConstReferenceFromSmartPointer(T ptr)
+    {
+        common::ReferenceSmartPointer<T>* impl = new common::ReferenceSmartPointer<T>();
+
+        ConstReference<typename T::element_type> res(impl);
+
+        impl->Swap(ptr);
+
+        return res;
+    }
+
+    /**
+     * Copy object and wrap it to make ignite::Reference instance.
+     *
+     * Template type 'T' should be copy-constructible.
+     *
+     * @param val Instance.
+     * @return Implementation defined value. User should not explicitly use the
+     *     returned value.
+     */
+    template<typename T>
+    Reference<T> MakeReferenceFromCopy(const T& val)
+    {
+        common::ReferenceOwningRawPointer<T>* impl = new common::ReferenceOwningRawPointer<T>(new T(val));
+
+        return Reference<T>(impl);
+    }
+
+    /**
+     * Copy object and wrap it to make ignite::ConstReference instance.
+     *
+     * Template type 'T' should be copy-constructible.
+     *
+     * @param val Instance.
+     * @return Implementation defined value. User should not explicitly use the
+     *     returned value.
+     */
+    template<typename T>
+    ConstReference<T> MakeConstReferenceFromCopy(const T& val)
+    {
+        common::ReferenceOwningRawPointer<T>* impl = new common::ReferenceOwningRawPointer<T>(new T(val));
+
+        return ConstReference<T>(impl);
+    }
+
+    /**
+     * Make ignite::Reference instance out of pointer and pass its ownership.
+     * Passed object deleted by Ignite when no longer needed.
+     *
+     * There are no requirements for the template type T.
+     *
+     * @param val Instance.
+     * @return Implementation defined value. User should not explicitly use the
+     *     returned value.
+     */
+    template<typename T>
+    Reference<T> MakeReferenceFromOwningPointer(T* val)
+    {
+        common::ReferenceOwningRawPointer<T>* impl = new common::ReferenceOwningRawPointer<T>(val);
+
+        return Reference<T>(impl);
+    }
+
+    /**
+     * Make ignite::ConstReference instance out of pointer and pass its ownership.
+     * Passed object deleted by Ignite when no longer needed.
+     *
+     * There are no requirements for the template type T.
+     *
+     * @param val Instance.
+     * @return Implementation defined value. User should not explicitly use the
+     *     returned value.
+     */
+    template<typename T>
+    ConstReference<T> MakeConstReferenceFromOwningPointer(T* val)
+    {
+        common::ReferenceOwningRawPointer<T>* impl = new common::ReferenceOwningRawPointer<T>(val);
+
+        return ConstReference<T>(impl);
+    }
+
+    /**
+     * Make ignite::Reference instance out of reference.
+     * Ignite do not manage passed object and does not affect its lifetime.
+     *
+     * There are no requirements for the template type T.
+     *
+     * @param val Reference.
+     * @return Implementation defined value. User should not explicitly use the
+     *     returned value.
+     */
+    template<typename T>
+    Reference<T> MakeReference(T& val)
+    {
+        common::ReferenceNonOwningRawPointer<T>* impl = new common::ReferenceNonOwningRawPointer<T>(&val);
+
+        return Reference<T>(impl);
+    }
+
+    /**
+     * Make ignite::Reference instance out of pointer.
+     * Ignite do not manage passed object and does not affect its lifetime.
+     *
+     * There are no requirements for the template type T.
+     *
+     * @param val Reference.
+     * @return Implementation defined value. User should not explicitly use the
+     *     returned value.
+     */
+    template<typename T>
+    Reference<T> MakeReference(T* val)
+    {
+        common::ReferenceNonOwningRawPointer<T>* impl = new common::ReferenceNonOwningRawPointer<T>(val);
+
+        return Reference<T>(impl);
+    }
+
+    /**
+     * Make ignite::ConstReference instance out of constant reference.
+     * Ignite do not manage passed object and does not affect its lifetime.
+     *
+     * There are no requirements for the template type T.
+     *
+     * @param val Reference.
+     * @return Implementation defined value. User should not explicitly use the
+     *     returned value.
+     */
+    template<typename T>
+    ConstReference<T> MakeConstReference(const T& val)
+    {
+        common::ConstReferenceNonOwningRawPointer<T>* impl = new common::ConstReferenceNonOwningRawPointer<T>(&val);
+
+        return ConstReference<T>(impl);
+    }
+
+    /**
+     * Make ignite::ConstReference instance out of constant pointer.
+     * Ignite do not manage passed object and does not affect its lifetime.
+     *
+     * There are no requirements for the template type T.
+     *
+     * @param val Reference.
+     * @return Implementation defined value. User should not explicitly use the
+     *     returned value.
+     */
+    template<typename T>
+    ConstReference<T> MakeConstReference(const T* val)
+    {
+        common::ConstReferenceNonOwningRawPointer<T>* impl = new common::ConstReferenceNonOwningRawPointer<T>(val);
+
+        return ConstReference<T>(impl);
+    }
+}
+
+#endif //_IGNITE_COMMON_REFERENCE
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/common/project/vs/common.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/project/vs/common.vcxproj b/modules/platforms/cpp/common/project/vs/common.vcxproj
index 149fa48..99fd551 100644
--- a/modules/platforms/cpp/common/project/vs/common.vcxproj
+++ b/modules/platforms/cpp/common/project/vs/common.vcxproj
@@ -171,10 +171,12 @@
     <ClInclude Include="..\..\include\ignite\common\dynamic_size_array.h" />
     <ClInclude Include="..\..\include\ignite\common\fixed_size_array.h" />
     <ClInclude Include="..\..\include\ignite\common\bits.h" />
+    <ClInclude Include="..\..\include\ignite\common\reference_impl.h" />
     <ClInclude Include="..\..\include\ignite\common\utils.h" />
     <ClInclude Include="..\..\include\ignite\date.h" />
     <ClInclude Include="..\..\include\ignite\guid.h" />
     <ClInclude Include="..\..\include\ignite\ignite_error.h" />
+    <ClInclude Include="..\..\include\ignite\reference.h" />
     <ClInclude Include="..\..\include\ignite\timestamp.h" />
     <ClInclude Include="..\..\os\win\include\ignite\common\common.h" />
     <ClInclude Include="..\..\os\win\include\ignite\common\concurrent_os.h" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/common/project/vs/common.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/project/vs/common.vcxproj.filters b/modules/platforms/cpp/common/project/vs/common.vcxproj.filters
index ae17daf..d99722b 100644
--- a/modules/platforms/cpp/common/project/vs/common.vcxproj.filters
+++ b/modules/platforms/cpp/common/project/vs/common.vcxproj.filters
@@ -58,6 +58,12 @@
     <ClInclude Include="..\..\include\ignite\common\decimal.h">
       <Filter>Code\common</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\include\ignite\common\reference_impl.h">
+      <Filter>Code\common</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\reference.h">
+      <Filter>Code</Filter>
+    </ClInclude>
   </ItemGroup>
   <ItemGroup>
     <ClCompile Include="..\..\src\date.cpp">

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core-test/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/Makefile.am b/modules/platforms/cpp/core-test/Makefile.am
index 42680cd..3cf1d0e 100644
--- a/modules/platforms/cpp/core-test/Makefile.am
+++ b/modules/platforms/cpp/core-test/Makefile.am
@@ -43,15 +43,20 @@ AM_CXXFLAGS = \
 
 ignite_tests_LDADD = \
     @top_srcdir@/core/libignite.la \
-    -lpthread
+    -lpthread \
+    -lboost_thread \
+    -lboost_system \
+    -lboost_chrono
 
 ignite_tests_LDFLAGS = \
     -static-libtool-libs
 
 ignite_tests_SOURCES = \
+    src/reference_test.cpp \
     src/bits_test.cpp \
     src/cache_test.cpp \
     src/cache_query_test.cpp \
+    src/continuous_query_test.cpp \
     src/concurrent_test.cpp \
     src/ignition_test.cpp \
     src/interop_memory_test.cpp \

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core-test/config/cache-query-continuous.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/config/cache-query-continuous.xml b/modules/platforms/cpp/core-test/config/cache-query-continuous.xml
new file mode 100644
index 0000000..1b940fd
--- /dev/null
+++ b/modules/platforms/cpp/core-test/config/cache-query-continuous.xml
@@ -0,0 +1,87 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="transactional_no_backup"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
+                    <property name="backups" value="0"/>
+                    <property name="startSize" value="10"/>
+                    <property name="queryEntities">
+                        <list>
+                            <bean class="org.apache.ignite.cache.QueryEntity">
+                                <property name="keyType" value="java.lang.Integer"/>
+                                <property name="valueType" value="TestEntry"/>
+
+                                <property name="fields">
+                                    <map>
+                                        <entry key="value" value="java.lang.Integer"/>
+                                    </map>
+                                </property>
+
+                                <property name="indexes">
+                                    <list>
+                                        <bean class="org.apache.ignite.cache.QueryIndex">
+                                            <property name="fields">
+                                                <map>
+                                                    <entry key="value" value="true"/>
+                                                </map>
+                                            </property>
+                                            <property name="indexType" value="FULLTEXT"/>
+                                        </bean>
+                                    </list>
+                                </property>
+                            </bean>
+                        </list>
+                    </property>
+                </bean>
+            </list>
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+                <property name="socketTimeout" value="300" />
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
index 6f13719..a41d8f8 100644
--- a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
+++ b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
@@ -43,6 +43,7 @@
     <ClCompile Include="..\..\src\decimal_test.cpp" />
     <ClCompile Include="..\..\src\dynamic_size_array_test.cpp" />
     <ClCompile Include="..\..\src\fixed_size_array_test.cpp" />
+    <ClCompile Include="..\..\src\continuous_query_test.cpp" />
     <ClCompile Include="..\..\src\ignite_error_test.cpp" />
     <ClCompile Include="..\..\src\ignition_test.cpp" />
     <ClCompile Include="..\..\src\handle_registry_test.cpp" />
@@ -54,6 +55,7 @@
     <ClCompile Include="..\..\src\interop_memory_test.cpp" />
     <ClCompile Include="..\..\src\interop_test.cpp" />
     <ClCompile Include="..\..\src\bits_test.cpp" />
+    <ClCompile Include="..\..\src\reference_test.cpp" />
     <ClCompile Include="..\..\src\teamcity_boost.cpp" />
     <ClCompile Include="..\..\src\teamcity_messages.cpp" />
     <ClCompile Include="..\..\src\transactions_test.cpp" />
@@ -129,7 +131,7 @@
     </ClCompile>
     <Link>
       <GenerateDebugInformation>true</GenerateDebugInformation>
-      <AdditionalDependencies>$(BOOST_HOME)\lib64-msvc-10.0\libboost_unit_test_framework-vc100-mt-gd-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalDependencies>$(BOOST_HOME)\lib64-msvc-10.0\libboost_unit_test_framework-vc100-mt-gd-1_58.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_thread-vc100-mt-gd-1_58.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_system-vc100-mt-gd-1_58.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_chrono-vc100-mt-gd-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
       <SubSystem>Console</SubSystem>
     </Link>
   </ItemDefinitionGroup>
@@ -144,7 +146,7 @@
     </ClCompile>
     <Link>
       <GenerateDebugInformation>true</GenerateDebugInformation>
-      <AdditionalDependencies>$(BOOST_HOME)\lib64-msvc-10.0\libboost_unit_test_framework-vc100-mt-gd-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalDependencies>$(BOOST_HOME)\lib32-msvc-10.0\libboost_unit_test_framework-vc100-mt-gd-1_58.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_thread-vc100-mt-gd-1_58.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_system-vc100-mt-gd-1_58.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_chrono-vc100-mt-gd-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
     </Link>
   </ItemDefinitionGroup>
   <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
@@ -162,7 +164,7 @@
       <GenerateDebugInformation>true</GenerateDebugInformation>
       <EnableCOMDATFolding>true</EnableCOMDATFolding>
       <OptimizeReferences>true</OptimizeReferences>
-      <AdditionalDependencies>$(BOOST_HOME)\lib64-msvc-10.0\libboost_unit_test_framework-vc100-mt-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalDependencies>$(BOOST_HOME)\lib64-msvc-10.0\libboost_unit_test_framework-vc100-mt-1_58.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_thread-vc100-mt-1_58.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_system-vc100-mt-1_58.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_chrono-vc100-mt-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
     </Link>
   </ItemDefinitionGroup>
   <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
@@ -180,7 +182,7 @@
       <GenerateDebugInformation>true</GenerateDebugInformation>
       <EnableCOMDATFolding>true</EnableCOMDATFolding>
       <OptimizeReferences>true</OptimizeReferences>
-      <AdditionalDependencies>$(BOOST_HOME)\lib64-msvc-10.0\libboost_unit_test_framework-vc100-mt-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalDependencies>$(BOOST_HOME)\lib32-msvc-10.0\libboost_unit_test_framework-vc100-mt-1_58.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_thread-vc100-mt-1_58.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_system-vc100-mt-1_58.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_chrono-vc100-mt-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
     </Link>
   </ItemDefinitionGroup>
   <Import Project="$(VCTargetsPath)\Microsoft.Cpp.targets" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
index cf1aaca..a95e3a4 100644
--- a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
+++ b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
@@ -58,6 +58,12 @@
     <ClCompile Include="..\..\src\interop_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\continuous_query_test.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
+    <ClCompile Include="..\..\src\reference_test.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\..\include\teamcity_messages.h">

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/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 c1c26ac..928d29e 100644
--- a/modules/platforms/cpp/core-test/src/cache_query_test.cpp
+++ b/modules/platforms/cpp/core-test/src/cache_query_test.cpp
@@ -232,8 +232,6 @@ private:
     int32_t someVal;
 };
 
-
-
 namespace ignite
 {
     namespace binary
@@ -296,74 +294,6 @@ namespace ignite
     }
 }
 
-///**
-// * Test setup fixture.
-// */
-//struct CacheQueryTestSuiteFixture
-//{
-//    Ignite StartNode(const char* name)
-//    {
-//        IgniteConfiguration cfg;
-//
-//        cfg.jvmOpts.push_back("-Xdebug");
-//        cfg.jvmOpts.push_back("-Xnoagent");
-//        cfg.jvmOpts.push_back("-Djava.compiler=NONE");
-//        cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005");
-//        cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError");
-//
-//#ifdef IGNITE_TESTS_32
-//        cfg.jvmInitMem = 256;
-//        cfg.jvmMaxMem = 768;
-//#else
-//        cfg.jvmInitMem = 1024;
-//        cfg.jvmMaxMem = 4096;
-//#endif
-//
-//        cfg.springCfgPath.assign(getenv("IGNITE_NATIVE_TEST_CPP_CONFIG_PATH")).append("/cache-query.xml");
-//
-//        IgniteError err;
-//
-//        Ignite grid0 = Ignition::Start(cfg, name, &err);
-//
-//        if (err.GetCode() != IgniteError::IGNITE_SUCCESS)
-//            BOOST_ERROR(err.GetText());
-//
-//        return grid0;
-//    }
-//
-//
-//    /**
-//     * Constructor.
-//     */
-//    CacheQueryTestSuiteFixture() : 
-//        grid(StartNode("Node1"))
-//    {
-//        // No-op.
-//    }
-//
-//    /**
-//     * Destructor.
-//     */
-//    ~CacheQueryTestSuiteFixture()
-//    {
-//        Ignition::StopAll(true);
-//    }
-//
-//    /** Person cache accessor. */
-//    Cache<int, QueryPerson> GetPersonCache()
-//    {
-//        return grid.GetCache<int, QueryPerson>("QueryPerson");
-//    }
-//
-    ///** Relation cache accessor. */
-    //Cache<int, QueryRelation> GetRelationCache()
-    //{
-    //    return grid.GetCache<int, QueryRelation>("QueryRelation");
-    //}
-//
-//    /** Node started during the test. */
-//    Ignite grid;
-//};
 
 /**
  * Count number of records returned by cursor.
@@ -677,7 +607,11 @@ struct CacheQueryTestSuiteFixture
         cfg.jvmMaxMem = 4096;
 #endif
 
-        cfg.springCfgPath.assign(getenv("IGNITE_NATIVE_TEST_CPP_CONFIG_PATH")).append("/cache-query.xml");
+        const char* cfgPath = getenv("IGNITE_NATIVE_TEST_CPP_CONFIG_PATH");
+
+        BOOST_CHECK(cfgPath != 0);
+
+        cfg.springCfgPath.assign(cfgPath).append("/cache-query.xml");
 
         IgniteError err;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core-test/src/continuous_query_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/continuous_query_test.cpp b/modules/platforms/cpp/core-test/src/continuous_query_test.cpp
new file mode 100644
index 0000000..e9d7e8a
--- /dev/null
+++ b/modules/platforms/cpp/core-test/src/continuous_query_test.cpp
@@ -0,0 +1,611 @@
+/*
+ * 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.
+ */
+
+#ifndef _MSC_VER
+    #define BOOST_TEST_DYN_LINK
+#endif
+
+#include <deque>
+
+#include <boost/test/unit_test.hpp>
+#include <boost/optional.hpp>
+#include <boost/thread/mutex.hpp>
+#include <boost/thread/condition_variable.hpp>
+
+#include "ignite/ignition.h"
+#include "ignite/cache/cache.h"
+
+using namespace ignite;
+using namespace ignite::cache;
+using namespace ignite::cache::event;
+using namespace ignite::cache::query;
+using namespace ignite::cache::query::continuous;
+using namespace boost::unit_test;
+
+/**
+ * Very simple concurrent queue implementation.
+ */
+template<typename T>
+class ConcurrentQueue
+{
+public:
+    /*
+     * Constructor.
+     */
+    ConcurrentQueue()
+    {
+        // No-op.
+    }
+
+    /*
+     * Push next element to queue.
+     *
+     * @param val Value to push.
+     */
+    void Push(const T& val)
+    {
+        boost::unique_lock<boost::mutex> guard(mutex);
+
+        queue.push_back(val);
+
+        cv.notify_one();
+    }
+
+    /*
+     * Pull element from the queue with the specified timeout.
+     *
+     * @param val Value is placed there on success.
+     * @param timeout Timeout.
+     * @return True on success and false on timeout.
+     */
+    template <typename Rep, typename Period>
+    bool Pull(T& val, const boost::chrono::duration<Rep, Period>& timeout)
+    {
+        boost::unique_lock<boost::mutex> guard(mutex);
+
+        if (queue.empty())
+        {
+            boost::cv_status res = cv.wait_for(guard, timeout);
+
+            if (res == boost::cv_status::timeout)
+                return false;
+        }
+
+        assert(!queue.empty());
+
+        val = queue.front();
+
+        queue.pop_front();
+
+        return true;
+    }
+
+private:
+    boost::mutex mutex;
+
+    boost::condition_variable cv;
+
+    std::deque<T> queue;
+};
+
+/*
+ * Test listener class. Stores events it has been notified about in concurrent
+ * queue so they can be checked later.
+ */
+template<typename K, typename V>
+class Listener : public CacheEntryEventListener<K, V>
+{
+public:
+    /*
+     * Default constructor.
+     */
+    Listener()
+    {
+        // No-op.
+    }
+
+    /**
+     * Event callback.
+     *
+     * @param evts Events.
+     * @param num Events number.
+     */
+    virtual void OnEvent(const CacheEntryEvent<K, V>* evts, uint32_t num)
+    {
+        for (uint32_t i = 0; i < num; ++i)
+            eventQueue.Push(evts[i]);
+    }
+
+    /*
+     * Check that next received event contains specific values.
+     *
+     * @param key Key.
+     * @param oldVal Old value.
+     * @param val Current value.
+     */
+    void CheckNextEvent(const K& key, boost::optional<V> oldVal, boost::optional<V> val)
+    {
+        CacheEntryEvent<K, V> event;
+        bool success = eventQueue.Pull(event, boost::chrono::seconds(1));
+
+        BOOST_REQUIRE(success);
+
+        BOOST_CHECK_EQUAL(event.GetKey(), key);
+        BOOST_CHECK_EQUAL(event.HasOldValue(), oldVal.is_initialized());
+        BOOST_CHECK_EQUAL(event.HasValue(), val.is_initialized());
+
+        if (oldVal && event.HasOldValue())
+            BOOST_CHECK_EQUAL(event.GetOldValue().value, oldVal->value);
+
+        if (val && event.HasValue())
+            BOOST_CHECK_EQUAL(event.GetValue().value, val->value);
+    }
+
+    /*
+     * Check that there is no event for the specified ammount of time.
+     *
+     * @param timeout Timeout.
+     */
+    template <typename Rep, typename Period>
+    void CheckNoEvent(const boost::chrono::duration<Rep, Period>& timeout)
+    {
+        CacheEntryEvent<K, V> event;
+        bool success = eventQueue.Pull(event, timeout);
+
+        BOOST_REQUIRE(!success);
+    }
+
+private:
+    // Events queue.
+    ConcurrentQueue< CacheEntryEvent<K, V> > eventQueue;
+};
+
+/*
+ * Test entry.
+ */
+struct TestEntry
+{
+    /*
+     * Default constructor.
+     */
+    TestEntry() : value(0)
+    {
+        // No-op.
+    }
+
+    /*
+     * Constructor.
+     */
+    TestEntry(int32_t val) : value(val)
+    {
+        // No-op.
+    }
+
+    /* Value */
+    int32_t value;
+};
+
+namespace ignite
+{
+    namespace binary
+    {
+        /**
+        * Binary type definition.
+        */
+        IGNITE_BINARY_TYPE_START(TestEntry)
+            IGNITE_BINARY_GET_TYPE_ID_AS_HASH(TestEntry)
+            IGNITE_BINARY_GET_TYPE_NAME_AS_IS(TestEntry)
+            IGNITE_BINARY_GET_FIELD_ID_AS_HASH
+            IGNITE_BINARY_GET_HASH_CODE_ZERO(TestEntry)
+            IGNITE_BINARY_IS_NULL_FALSE(TestEntry)
+            IGNITE_BINARY_GET_NULL_DEFAULT_CTOR(TestEntry)
+
+            void Write(BinaryWriter& writer, const TestEntry& obj)
+            {
+                writer.WriteInt32("value", obj.value);
+            }
+
+            TestEntry Read(BinaryReader& reader)
+            {
+                TestEntry res;
+                res.value = reader.ReadInt32("value");
+
+                return res;
+            }
+
+        IGNITE_BINARY_TYPE_END
+    }
+}
+
+/*
+ * Test setup fixture.
+ */
+struct ContinuousQueryTestSuiteFixture
+{
+    Ignite grid;
+
+    Cache<int, TestEntry> cache;
+
+    /*
+     * Get configuration for nodes.
+     */
+    IgniteConfiguration GetConfiguration()
+    {
+        IgniteConfiguration cfg;
+
+        cfg.jvmOpts.push_back("-Xdebug");
+        cfg.jvmOpts.push_back("-Xnoagent");
+        cfg.jvmOpts.push_back("-Djava.compiler=NONE");
+        cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005");
+        cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError");
+
+#ifdef IGNITE_TESTS_32
+        cfg.jvmInitMem = 256;
+        cfg.jvmMaxMem = 768;
+#else
+        cfg.jvmInitMem = 1024;
+        cfg.jvmMaxMem = 4096;
+#endif
+
+        char* cfgPath = getenv("IGNITE_NATIVE_TEST_CPP_CONFIG_PATH");
+
+        cfg.springCfgPath = std::string(cfgPath).append("/").append("cache-query-continuous.xml");
+
+        return cfg;
+    }
+
+    /*
+     * Constructor.
+     */
+    ContinuousQueryTestSuiteFixture() :
+        grid(Ignition::Start(GetConfiguration(), "node-01")),
+        cache(grid.GetCache<int, TestEntry>("transactional_no_backup"))
+    {
+        // No-op.
+    }
+
+    /*
+     * Destructor.
+     */
+    ~ContinuousQueryTestSuiteFixture()
+    {
+        Ignition::StopAll(false);
+
+        grid = Ignite();
+    }
+};
+
+void CheckEvents(Cache<int, TestEntry>& cache, Listener<int, TestEntry>& lsnr)
+{
+    cache.Put(1, TestEntry(10));
+    lsnr.CheckNextEvent(1, boost::none, TestEntry(10));
+
+    cache.Put(1, TestEntry(20));
+    lsnr.CheckNextEvent(1, TestEntry(10), TestEntry(20));
+
+    cache.Put(2, TestEntry(20));
+    lsnr.CheckNextEvent(2, boost::none, TestEntry(20));
+
+    cache.Remove(1);
+    lsnr.CheckNextEvent(1, TestEntry(20), boost::none);
+}
+
+BOOST_FIXTURE_TEST_SUITE(ContinuousQueryTestSuite, ContinuousQueryTestSuiteFixture)
+
+BOOST_AUTO_TEST_CASE(TestBasic)
+{
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry);
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestInitialQueryScan)
+{
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    cache.Put(11, TestEntry(111));
+    cache.Put(22, TestEntry(222));
+    cache.Put(33, TestEntry(333));
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry, ScanQuery());
+
+    std::vector< CacheEntry<int, TestEntry> > vals;
+
+    handle.GetInitialQueryCursor().GetAll(vals);
+
+    BOOST_CHECK_THROW(handle.GetInitialQueryCursor(), IgniteError);
+
+    BOOST_REQUIRE_EQUAL(vals.size(), 3);
+
+    BOOST_CHECK_EQUAL(vals[0].GetKey(), 11);
+    BOOST_CHECK_EQUAL(vals[1].GetKey(), 22);
+    BOOST_CHECK_EQUAL(vals[2].GetKey(), 33);
+
+    BOOST_CHECK_EQUAL(vals[0].GetValue().value, 111);
+    BOOST_CHECK_EQUAL(vals[1].GetValue().value, 222);
+    BOOST_CHECK_EQUAL(vals[2].GetValue().value, 333);
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestInitialQuerySql)
+{
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    cache.Put(11, TestEntry(111));
+    cache.Put(22, TestEntry(222));
+    cache.Put(33, TestEntry(333));
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry, SqlQuery("TestEntry", "value > 200"));
+
+    std::vector< CacheEntry<int, TestEntry> > vals;
+
+    handle.GetInitialQueryCursor().GetAll(vals);
+
+    BOOST_CHECK_THROW(handle.GetInitialQueryCursor(), IgniteError);
+
+    BOOST_REQUIRE_EQUAL(vals.size(), 2);
+
+    BOOST_CHECK_EQUAL(vals[0].GetKey(), 22);
+    BOOST_CHECK_EQUAL(vals[1].GetKey(), 33);
+
+    BOOST_CHECK_EQUAL(vals[0].GetValue().value, 222);
+    BOOST_CHECK_EQUAL(vals[1].GetValue().value, 333);
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestInitialQueryText)
+{
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    cache.Put(11, TestEntry(111));
+    cache.Put(22, TestEntry(222));
+    cache.Put(33, TestEntry(333));
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry, TextQuery("TestEntry", "222"));
+
+    std::vector< CacheEntry<int, TestEntry> > vals;
+
+    handle.GetInitialQueryCursor().GetAll(vals);
+
+    BOOST_CHECK_THROW(handle.GetInitialQueryCursor(), IgniteError);
+
+    BOOST_REQUIRE_EQUAL(vals.size(), 1);
+
+    BOOST_CHECK_EQUAL(vals[0].GetKey(), 22);
+
+    BOOST_CHECK_EQUAL(vals[0].GetValue().value, 222);
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestBasicNoExcept)
+{
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    IgniteError err;
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry, err);
+
+    BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_SUCCESS);
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestInitialQueryScanNoExcept)
+{
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    cache.Put(11, TestEntry(111));
+    cache.Put(22, TestEntry(222));
+    cache.Put(33, TestEntry(333));
+
+    IgniteError err;
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry, ScanQuery(), err);
+
+    BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_SUCCESS);
+
+    std::vector< CacheEntry<int, TestEntry> > vals;
+
+    handle.GetInitialQueryCursor().GetAll(vals);
+
+    BOOST_CHECK_THROW(handle.GetInitialQueryCursor(), IgniteError);
+
+    BOOST_REQUIRE_EQUAL(vals.size(), 3);
+
+    BOOST_CHECK_EQUAL(vals[0].GetKey(), 11);
+    BOOST_CHECK_EQUAL(vals[1].GetKey(), 22);
+    BOOST_CHECK_EQUAL(vals[2].GetKey(), 33);
+
+    BOOST_CHECK_EQUAL(vals[0].GetValue().value, 111);
+    BOOST_CHECK_EQUAL(vals[1].GetValue().value, 222);
+    BOOST_CHECK_EQUAL(vals[2].GetValue().value, 333);
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestInitialQuerySqlNoExcept)
+{
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    cache.Put(11, TestEntry(111));
+    cache.Put(22, TestEntry(222));
+    cache.Put(33, TestEntry(333));
+
+    IgniteError err;
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry, SqlQuery("TestEntry", "value > 200"), err);
+
+    BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_SUCCESS);
+
+    std::vector< CacheEntry<int, TestEntry> > vals;
+
+    handle.GetInitialQueryCursor().GetAll(vals);
+
+    BOOST_CHECK_THROW(handle.GetInitialQueryCursor(), IgniteError);
+
+    BOOST_REQUIRE_EQUAL(vals.size(), 2);
+
+    BOOST_CHECK_EQUAL(vals[0].GetKey(), 22);
+    BOOST_CHECK_EQUAL(vals[1].GetKey(), 33);
+
+    BOOST_CHECK_EQUAL(vals[0].GetValue().value, 222);
+    BOOST_CHECK_EQUAL(vals[1].GetValue().value, 333);
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestInitialQueryTextNoExcept)
+{
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    cache.Put(11, TestEntry(111));
+    cache.Put(22, TestEntry(222));
+    cache.Put(33, TestEntry(333));
+
+    IgniteError err;
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry, TextQuery("TestEntry", "222"), err);
+
+    BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_SUCCESS);
+
+    std::vector< CacheEntry<int, TestEntry> > vals;
+
+    handle.GetInitialQueryCursor().GetAll(vals);
+
+    BOOST_CHECK_THROW(handle.GetInitialQueryCursor(), IgniteError);
+
+    BOOST_REQUIRE_EQUAL(vals.size(), 1);
+
+    BOOST_CHECK_EQUAL(vals[0].GetKey(), 22);
+
+    BOOST_CHECK_EQUAL(vals[0].GetValue().value, 222);
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestExpiredQuery)
+{
+    Listener<int, TestEntry> lsnr;
+    ContinuousQueryHandle<int, TestEntry> handle;
+
+    {
+        // Query scope.
+        ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+        handle = cache.QueryContinuous(qry);
+    }
+
+    // Query is destroyed here.
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestSetGetLocal)
+{
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    BOOST_CHECK(!qry.GetLocal());
+
+    qry.SetLocal(true);
+
+    BOOST_CHECK(qry.GetLocal());
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry);
+
+    BOOST_CHECK(qry.GetLocal());
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestGetSetBufferSize)
+{
+    typedef ContinuousQuery<int, TestEntry> QueryType;
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    BOOST_CHECK_EQUAL(qry.GetBufferSize(), QueryType::DEFAULT_BUFFER_SIZE);
+
+    qry.SetBufferSize(2 * QueryType::DEFAULT_BUFFER_SIZE);
+
+    BOOST_CHECK_EQUAL(qry.GetBufferSize(), 2 * QueryType::DEFAULT_BUFFER_SIZE);
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry);
+
+    BOOST_CHECK_EQUAL(qry.GetBufferSize(), 2 * QueryType::DEFAULT_BUFFER_SIZE);
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestGetSetTimeInterval)
+{
+    typedef ContinuousQuery<int, TestEntry> QueryType;
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    qry.SetBufferSize(10);
+
+    BOOST_CHECK_EQUAL(qry.GetTimeInterval(), static_cast<int>(QueryType::DEFAULT_TIME_INTERVAL));
+
+    qry.SetTimeInterval(500);
+
+    BOOST_CHECK_EQUAL(qry.GetTimeInterval(), 500);
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry);
+
+    BOOST_CHECK_EQUAL(qry.GetTimeInterval(), 500);
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestPublicPrivateConstantsConsistence)
+{
+    typedef ContinuousQuery<int, TestEntry> QueryType;
+    typedef impl::cache::query::continuous::ContinuousQueryImpl<int, TestEntry> QueryImplType;
+    
+    BOOST_CHECK_EQUAL(static_cast<int>(QueryImplType::DEFAULT_TIME_INTERVAL),
+        static_cast<int>(QueryType::DEFAULT_TIME_INTERVAL));
+
+    BOOST_CHECK_EQUAL(static_cast<int>(QueryImplType::DEFAULT_BUFFER_SIZE),
+        static_cast<int>(QueryType::DEFAULT_BUFFER_SIZE));
+}
+
+BOOST_AUTO_TEST_SUITE_END()
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core-test/src/handle_registry_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/handle_registry_test.cpp b/modules/platforms/cpp/core-test/src/handle_registry_test.cpp
index bc4a654..0956d9b 100644
--- a/modules/platforms/cpp/core-test/src/handle_registry_test.cpp
+++ b/modules/platforms/cpp/core-test/src/handle_registry_test.cpp
@@ -36,7 +36,7 @@ struct HandleRegistryTestProbe
     }
 };
 
-class HandleRegistryTestEntry : public HandleRegistryEntry
+class HandleRegistryTestEntry
 {
 public:
     HandleRegistryTestEntry(HandleRegistryTestProbe* probe) : probe(probe)
@@ -67,9 +67,9 @@ BOOST_AUTO_TEST_CASE(TestCritical)
     HandleRegistryTestEntry* entry1 = new HandleRegistryTestEntry(&probe1);
     HandleRegistryTestEntry* entry2 = new HandleRegistryTestEntry(&probe2);
 
-    int64_t hnd0 = reg.AllocateCritical(SharedPointer<HandleRegistryEntry>(entry0));
-    int64_t hnd1 = reg.AllocateCritical(SharedPointer<HandleRegistryEntry>(entry1));
-    int64_t hnd2 = reg.AllocateCritical(SharedPointer<HandleRegistryEntry>(entry2));
+    int64_t hnd0 = reg.AllocateCritical(SharedPointer<HandleRegistryTestEntry>(entry0));
+    int64_t hnd1 = reg.AllocateCritical(SharedPointer<HandleRegistryTestEntry>(entry1));
+    int64_t hnd2 = reg.AllocateCritical(SharedPointer<HandleRegistryTestEntry>(entry2));
 
     BOOST_REQUIRE(reg.Get(hnd0).Get() == entry0);
     BOOST_REQUIRE(!probe0.deleted);
@@ -109,7 +109,7 @@ BOOST_AUTO_TEST_CASE(TestCritical)
     HandleRegistryTestProbe closedProbe;
     HandleRegistryTestEntry* closedEntry = new HandleRegistryTestEntry(&closedProbe);
 
-    int64_t closedHnd = closedReg.AllocateCritical(SharedPointer<HandleRegistryEntry>(closedEntry));
+    int64_t closedHnd = closedReg.AllocateCritical(SharedPointer<HandleRegistryTestEntry>(closedEntry));
     BOOST_REQUIRE(closedHnd == -1);
     BOOST_REQUIRE(closedProbe.deleted);
 }
@@ -126,9 +126,9 @@ BOOST_AUTO_TEST_CASE(TestNonCritical)
     HandleRegistryTestEntry* entry1 = new HandleRegistryTestEntry(&probe1);
     HandleRegistryTestEntry* entry2 = new HandleRegistryTestEntry(&probe2);
 
-    int64_t hnd0 = reg.AllocateCritical(SharedPointer<HandleRegistryEntry>(entry0));
-    int64_t hnd1 = reg.Allocate(SharedPointer<HandleRegistryEntry>(entry1));
-    int64_t hnd2 = reg.Allocate(SharedPointer<HandleRegistryEntry>(entry2));
+    int64_t hnd0 = reg.AllocateCritical(SharedPointer<HandleRegistryTestEntry>(entry0));
+    int64_t hnd1 = reg.Allocate(SharedPointer<HandleRegistryTestEntry>(entry1));
+    int64_t hnd2 = reg.Allocate(SharedPointer<HandleRegistryTestEntry>(entry2));
 
     BOOST_REQUIRE(reg.Get(hnd0).Get() == entry0);
     BOOST_REQUIRE(!probe0.deleted);
@@ -168,7 +168,7 @@ BOOST_AUTO_TEST_CASE(TestNonCritical)
     HandleRegistryTestProbe closedProbe;
     HandleRegistryTestEntry* closedEntry = new HandleRegistryTestEntry(&closedProbe);
 
-    int64_t closedHnd = closedReg.Allocate(SharedPointer<HandleRegistryEntry>(closedEntry));
+    int64_t closedHnd = closedReg.Allocate(SharedPointer<HandleRegistryTestEntry>(closedEntry));
     BOOST_REQUIRE(closedHnd == -1);
     BOOST_REQUIRE(closedProbe.deleted);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core-test/src/reference_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/reference_test.cpp b/modules/platforms/cpp/core-test/src/reference_test.cpp
new file mode 100644
index 0000000..f5c3e8d
--- /dev/null
+++ b/modules/platforms/cpp/core-test/src/reference_test.cpp
@@ -0,0 +1,412 @@
+/*
+ * 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.
+ */
+
+#ifndef _MSC_VER
+    #define BOOST_TEST_DYN_LINK
+#endif
+
+#include <memory>
+
+#include <boost/test/unit_test.hpp>
+#include <boost/smart_ptr.hpp>
+#include <boost/interprocess/smart_ptr/unique_ptr.hpp>
+
+#include <ignite/reference.h>
+
+using namespace ignite;
+using namespace boost::unit_test;
+
+class LivenessMarker
+{
+public:
+    LivenessMarker(bool& flag) :
+        flag(flag)
+    {
+        flag = true;
+    }
+
+    LivenessMarker(const LivenessMarker& other) :
+        flag(other.flag)
+    {
+        // No-op.
+    }
+
+    LivenessMarker& operator=(const LivenessMarker& other)
+    {
+        flag = other.flag;
+
+        return *this;
+    }
+
+    ~LivenessMarker()
+    {
+        flag = false;
+    }
+
+private:
+    bool& flag;
+};
+
+class InstanceCounter
+{
+public:
+    InstanceCounter(int& counter) :
+        counter(&counter)
+    {
+        ++(*this->counter);
+    }
+
+    InstanceCounter(const InstanceCounter& other) :
+        counter(other.counter)
+    {
+        ++(*counter);
+    }
+
+    InstanceCounter& operator=(const InstanceCounter& other)
+    {
+        counter = other.counter;
+
+        ++(*counter);
+
+        return *this;
+    }
+
+    ~InstanceCounter()
+    {
+        --(*counter);
+    }
+
+private:
+    int* counter;
+};
+
+
+void TestFunction(Reference<LivenessMarker> ptr)
+{
+    Reference<LivenessMarker> copy(ptr);
+    Reference<LivenessMarker> copy2(ptr);
+}
+
+struct C1
+{
+    int c1;
+};
+
+struct C2
+{
+    int c2;
+};
+
+struct C3 : C1, C2
+{
+    int c3;
+};
+
+void TestFunction1(Reference<C1> c1, int expected)
+{
+    BOOST_CHECK_EQUAL(c1.Get().c1, expected);
+}
+
+void TestFunction2(Reference<C2> c2, int expected)
+{
+    BOOST_CHECK_EQUAL(c2.Get().c2, expected);
+}
+
+void TestFunction3(Reference<C3> c3, int expected)
+{
+    BOOST_CHECK_EQUAL(c3.Get().c3, expected);
+}
+
+void TestFunctionConst1(ConstReference<C1> c1, int expected)
+{
+    BOOST_CHECK_EQUAL(c1.Get().c1, expected);
+}
+
+void TestFunctionConst2(ConstReference<C2> c2, int expected)
+{
+    BOOST_CHECK_EQUAL(c2.Get().c2, expected);
+}
+
+void TestFunctionConst3(ConstReference<C3> c3, int expected)
+{
+    BOOST_CHECK_EQUAL(c3.Get().c3, expected);
+}
+
+BOOST_AUTO_TEST_SUITE(ReferenceTestSuite)
+
+BOOST_AUTO_TEST_CASE(StdSharedPointerTestBefore)
+{
+    bool objAlive = false;
+
+    std::shared_ptr<LivenessMarker> shared = std::make_shared<LivenessMarker>(objAlive);
+
+    BOOST_CHECK(objAlive);
+
+    {
+        Reference<LivenessMarker> smart = MakeReferenceFromSmartPointer(shared);
+
+        BOOST_CHECK(objAlive);
+
+        shared.reset();
+
+        BOOST_CHECK(objAlive);
+    }
+
+    BOOST_CHECK(!objAlive);
+}
+
+BOOST_AUTO_TEST_CASE(StdSharedPointerTestAfter)
+{
+    bool objAlive = false;
+
+    std::shared_ptr<LivenessMarker> shared = std::make_shared<LivenessMarker>(objAlive);
+
+    BOOST_CHECK(objAlive);
+
+    {
+        Reference<LivenessMarker> smart = MakeReferenceFromSmartPointer(shared);
+
+        BOOST_CHECK(objAlive);
+    }
+
+    BOOST_CHECK(objAlive);
+
+    shared.reset();
+
+    BOOST_CHECK(!objAlive);
+}
+
+BOOST_AUTO_TEST_CASE(StdAutoPointerTest)
+{
+    bool objAlive = false;
+
+    std::auto_ptr<LivenessMarker> autop(new LivenessMarker(objAlive));
+
+    BOOST_CHECK(objAlive);
+
+    {
+        Reference<LivenessMarker> smart = MakeReferenceFromSmartPointer(autop);
+
+        BOOST_CHECK(objAlive);
+    }
+
+    BOOST_CHECK(!objAlive);
+}
+
+BOOST_AUTO_TEST_CASE(StdUniquePointerTest)
+{
+    bool objAlive = false;
+
+    std::unique_ptr<LivenessMarker> unique(new LivenessMarker(objAlive));
+
+    BOOST_CHECK(objAlive);
+
+    {
+        Reference<LivenessMarker> smart = MakeReferenceFromSmartPointer(std::move(unique));
+
+        BOOST_CHECK(objAlive);
+    }
+
+    BOOST_CHECK(!objAlive);
+}
+
+BOOST_AUTO_TEST_CASE(BoostSharedPointerTestBefore)
+{
+    bool objAlive = false;
+
+    boost::shared_ptr<LivenessMarker> shared = boost::make_shared<LivenessMarker>(objAlive);
+
+    BOOST_CHECK(objAlive);
+
+    {
+        Reference<LivenessMarker> smart = MakeReferenceFromSmartPointer(shared);
+
+        BOOST_CHECK(objAlive);
+
+        shared.reset();
+
+        BOOST_CHECK(objAlive);
+    }
+
+    BOOST_CHECK(!objAlive);
+}
+
+BOOST_AUTO_TEST_CASE(BoostSharedPointerTestAfter)
+{
+    bool objAlive = false;
+
+    boost::shared_ptr<LivenessMarker> shared = boost::make_shared<LivenessMarker>(objAlive);
+
+    BOOST_CHECK(objAlive);
+
+    {
+        Reference<LivenessMarker> smart = MakeReferenceFromSmartPointer(shared);
+
+        BOOST_CHECK(objAlive);
+    }
+
+    BOOST_CHECK(objAlive);
+
+    shared.reset();
+
+    BOOST_CHECK(!objAlive);
+}
+
+BOOST_AUTO_TEST_CASE(PassingToFunction)
+{
+    bool objAlive = false;
+
+    std::shared_ptr<LivenessMarker> stdShared = std::make_shared<LivenessMarker>(objAlive);
+    std::unique_ptr<LivenessMarker> stdUnique(new LivenessMarker(objAlive));
+    std::auto_ptr<LivenessMarker> stdAuto(new LivenessMarker(objAlive));
+
+    boost::shared_ptr<LivenessMarker> boostShared = boost::make_shared<LivenessMarker>(objAlive);
+
+    TestFunction(MakeReferenceFromSmartPointer(stdShared));
+    TestFunction(MakeReferenceFromSmartPointer(std::move(stdUnique)));
+    TestFunction(MakeReferenceFromSmartPointer(stdAuto));
+
+    TestFunction(MakeReferenceFromSmartPointer(boostShared));
+}
+
+BOOST_AUTO_TEST_CASE(CopyTest)
+{
+    int instances = 0;
+
+    {
+        InstanceCounter counter(instances);
+
+        BOOST_CHECK_EQUAL(instances, 1);
+
+        {
+            Reference<InstanceCounter> copy = MakeReferenceFromCopy(counter);
+
+            BOOST_CHECK_EQUAL(instances, 2);
+        }
+
+        BOOST_CHECK_EQUAL(instances, 1);
+    }
+
+    BOOST_CHECK_EQUAL(instances, 0);
+}
+
+BOOST_AUTO_TEST_CASE(OwningPointerTest)
+{
+    int instances = 0;
+
+    {
+        InstanceCounter *counter = new InstanceCounter(instances);
+
+        BOOST_CHECK_EQUAL(instances, 1);
+
+        {
+            Reference<InstanceCounter> owned = MakeReferenceFromOwningPointer(counter);
+
+            BOOST_CHECK_EQUAL(instances, 1);
+        }
+
+        BOOST_CHECK_EQUAL(instances, 0);
+    }
+
+    BOOST_CHECK_EQUAL(instances, 0);
+}
+
+BOOST_AUTO_TEST_CASE(NonOwningPointerTest1)
+{
+    int instances = 0;
+
+    {
+        InstanceCounter counter(instances);
+
+        BOOST_CHECK_EQUAL(instances, 1);
+
+        {
+            Reference<InstanceCounter> copy = MakeReference(counter);
+
+            BOOST_CHECK_EQUAL(instances, 1);
+        }
+
+        BOOST_CHECK_EQUAL(instances, 1);
+    }
+
+    BOOST_CHECK_EQUAL(instances, 0);
+}
+
+BOOST_AUTO_TEST_CASE(NonOwningPointerTest2)
+{
+    int instances = 0;
+
+    InstanceCounter* counter = new InstanceCounter(instances);
+
+    BOOST_CHECK_EQUAL(instances, 1);
+
+    {
+        Reference<InstanceCounter> copy = MakeReference(*counter);
+
+        BOOST_CHECK_EQUAL(instances, 1);
+
+        delete counter;
+
+        BOOST_CHECK_EQUAL(instances, 0);
+    }
+
+    BOOST_CHECK_EQUAL(instances, 0);
+}
+
+BOOST_AUTO_TEST_CASE(CastTest)
+{
+    C3 testVal;
+
+    testVal.c1 = 1;
+    testVal.c2 = 2;
+    testVal.c3 = 3;
+
+    TestFunction1(MakeReference(testVal), 1);
+    TestFunction2(MakeReference(testVal), 2);
+    TestFunction3(MakeReference(testVal), 3);
+
+    TestFunction1(MakeReferenceFromCopy(testVal), 1);
+    TestFunction2(MakeReferenceFromCopy(testVal), 2);
+    TestFunction3(MakeReferenceFromCopy(testVal), 3);
+}
+
+BOOST_AUTO_TEST_CASE(ConstTest)
+{
+    C3 testVal;
+
+    testVal.c1 = 1;
+    testVal.c2 = 2;
+    testVal.c3 = 3;
+
+    TestFunctionConst1(MakeConstReference(testVal), 1);
+    TestFunctionConst2(MakeConstReference(testVal), 2);
+    TestFunctionConst3(MakeConstReference(testVal), 3);
+
+    TestFunctionConst1(MakeConstReferenceFromCopy(testVal), 1);
+    TestFunctionConst2(MakeConstReferenceFromCopy(testVal), 2);
+    TestFunctionConst3(MakeConstReferenceFromCopy(testVal), 3);
+
+    TestFunctionConst1(MakeReference(testVal), 1);
+    TestFunctionConst2(MakeReference(testVal), 2);
+    TestFunctionConst3(MakeReference(testVal), 3);
+
+    TestFunctionConst1(MakeReferenceFromCopy(testVal), 1);
+    TestFunctionConst2(MakeReferenceFromCopy(testVal), 2);
+    TestFunctionConst3(MakeReferenceFromCopy(testVal), 3);
+}
+
+BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/Makefile.am b/modules/platforms/cpp/core/Makefile.am
index 97523cf..758e68d 100644
--- a/modules/platforms/cpp/core/Makefile.am
+++ b/modules/platforms/cpp/core/Makefile.am
@@ -59,6 +59,7 @@ libignite_la_SOURCES = \
     src/impl/ignite_environment.cpp \
     src/impl/binary/binary_type_updater_impl.cpp \
     src/impl/handle_registry.cpp \
+    src/impl/cache/query/continuous/continuous_query_handle_impl.cpp \
     src/impl/cache/query/query_impl.cpp \
     src/impl/cache/cache_impl.cpp \
     src/impl/cache/query/query_batch.cpp \