You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by yz...@apache.org on 2017/07/28 12:07:46 UTC

[34/50] [abbrv] ignite git commit: IGNITE-4648 IgniteInternalTx.prepare() does not wait for async operations to complete

IGNITE-4648 IgniteInternalTx.prepare() does not wait for async operations to complete


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

Branch: refs/heads/ignite-5658
Commit: e96225f1ed4080243844108e798e3fa74d13d542
Parents: de259ff
Author: Dmitrii Ryabov <So...@gmail.com>
Authored: Wed Jul 26 17:00:52 2017 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Jul 26 17:00:52 2017 +0300

----------------------------------------------------------------------
 .../cache/GridCacheSharedContext.java           |   6 +-
 .../cache/distributed/near/GridNearTxLocal.java |   6 +-
 .../IgniteTxImplicitSingleStateImpl.java        |   2 +-
 .../IgniteTxRemoteStateAdapter.java             |   2 +-
 .../cache/transactions/IgniteTxState.java       |   2 +-
 .../cache/transactions/IgniteTxStateImpl.java   |   2 +-
 .../transactions/PlatformTransactions.java      |   2 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |   2 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |   8 +-
 .../processors/cache/jta/CacheJtaResource.java  |  20 +-
 .../cache/AbstractCacheJtaSelfTest.java         | 183 --------------
 .../GridPartitionedCacheJtaFactorySelfTest.java |  41 ---
 ...rtitionedCacheJtaFactoryUseSyncSelfTest.java |  32 ---
 ...titionedCacheJtaLookupClassNameSelfTest.java |  83 -------
 .../GridReplicatedCacheJtaFactorySelfTest.java  |  32 ---
 ...eplicatedCacheJtaFactoryUseSyncSelfTest.java |  32 ---
 ...plicatedCacheJtaLookupClassNameSelfTest.java |  32 ---
 .../cache/jta/AbstractCacheJtaSelfTest.java     | 248 +++++++++++++++++++
 .../GridPartitionedCacheJtaFactorySelfTest.java |  41 +++
 ...rtitionedCacheJtaFactoryUseSyncSelfTest.java |  32 +++
 ...titionedCacheJtaLookupClassNameSelfTest.java |  83 +++++++
 .../GridReplicatedCacheJtaFactorySelfTest.java  |  32 +++
 ...eplicatedCacheJtaFactoryUseSyncSelfTest.java |  32 +++
 ...plicatedCacheJtaLookupClassNameSelfTest.java |  32 +++
 .../processors/cache/jta/package-info.java      |  22 ++
 .../ignite/testsuites/IgniteJtaTestSuite.java   |  12 +-
 .../Cache/CacheAbstractTransactionalTest.cs     |  53 ++--
 27 files changed, 591 insertions(+), 483 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index 5387cc8..1876023 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -910,7 +910,7 @@ public class GridCacheSharedContext<K, V> {
      * @throws IgniteCheckedException If failed.
      */
     public void endTx(GridNearTxLocal tx) throws IgniteCheckedException {
-        tx.txState().awaitLastFut(this);
+        tx.txState().awaitLastFuture(this);
 
         tx.close();
     }
@@ -924,7 +924,7 @@ public class GridCacheSharedContext<K, V> {
         GridCacheContext ctx = tx.txState().singleCacheContext(this);
 
         if (ctx == null) {
-            tx.txState().awaitLastFut(this);
+            tx.txState().awaitLastFuture(this);
 
             return tx.commitNearTxLocalAsync();
         }
@@ -938,7 +938,7 @@ public class GridCacheSharedContext<K, V> {
      * @return Rollback future.
      */
     public IgniteInternalFuture rollbackTxAsync(GridNearTxLocal tx) throws IgniteCheckedException {
-        tx.txState().awaitLastFut(this);
+        tx.txState().awaitLastFuture(this);
 
         return tx.rollbackNearTxLocalAsync();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/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 81e5ca8..58ecee9 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
@@ -3120,9 +3120,13 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements AutoClosea
     }
 
     /**
+     * @param awaitLastFuture If true - method will wait until transaction finish every action started before.
      * @throws IgniteCheckedException If failed.
      */
-    public final void prepare() throws IgniteCheckedException {
+    public final void prepare(boolean awaitLastFuture) throws IgniteCheckedException {
+        if (awaitLastFuture)
+            txState().awaitLastFuture(cctx);
+
         prepareAsync().get();
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java
index 7610d50..886d0d6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java
@@ -91,7 +91,7 @@ public class IgniteTxImplicitSingleStateImpl extends IgniteTxLocalStateAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void awaitLastFut(GridCacheSharedContext ctx) {
+    @Override public void awaitLastFuture(GridCacheSharedContext ctx) {
         if (cacheCtx == null)
             return;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateAdapter.java
index 86ae684..bcb900c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateAdapter.java
@@ -44,7 +44,7 @@ public abstract class IgniteTxRemoteStateAdapter implements IgniteTxRemoteState
     }
 
     /** {@inheritDoc} */
-    @Override public void awaitLastFut(GridCacheSharedContext cctx) {
+    @Override public void awaitLastFuture(GridCacheSharedContext cctx) {
         assert false;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxState.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxState.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxState.java
index ee48ed7..1fe0d2a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxState.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxState.java
@@ -59,7 +59,7 @@ public interface IgniteTxState {
     /**
      * @param cctx Awaits for previous async operations on active caches to be completed.
      */
-    public void awaitLastFut(GridCacheSharedContext cctx);
+    public void awaitLastFuture(GridCacheSharedContext cctx);
 
     /**
      * @param cctx Context.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java
index ad4ca61..4f14b5c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java
@@ -107,7 +107,7 @@ public class IgniteTxStateImpl extends IgniteTxLocalStateAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void awaitLastFut(GridCacheSharedContext cctx) {
+    @Override public void awaitLastFuture(GridCacheSharedContext cctx) {
         for (int i = 0; i < activeCacheIds.size(); i++) {
             int cacheId = activeCacheIds.get(i);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
index 8f34343..8baca9b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
@@ -160,7 +160,7 @@ public class PlatformTransactions extends PlatformAbstractTarget {
     @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
         switch (type) {
             case OP_PREPARE:
-                ((TransactionProxyImpl)tx(val)).tx().prepare();
+                ((TransactionProxyImpl)tx(val)).tx().prepare(true);
 
                 return TRUE;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest.java
index 0463c46..7514555 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest.java
@@ -349,7 +349,7 @@ public abstract class IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest
             assertTrue(txEx.pessimistic());
 
             if (commmit) {
-                txEx.prepare();
+                txEx.prepare(true);
 
                 // Fail the node in the middle of transaction.
                 info(">>> Stopping primary node " + primaryNode);

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
index 06fbe8f..256e8da 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
@@ -152,7 +152,7 @@ public class GridCachePartitionedTxSalvageSelfTest extends GridCommonAbstractTes
      * Check whether caches has no transactions after salvage timeout.
      *
      * @param mode Transaction mode (PESSIMISTIC, OPTIMISTIC).
-     * @param prepare Whether to prepare transaction state (i.e. call {@link GridNearTxLocal#prepare()}).
+     * @param prepare Whether to prepare transaction state (i.e. call {@link GridNearTxLocal#prepare(boolean)}).
      * @throws Exception If failed.
      */
     private void checkSalvageAfterTimeout(TransactionConcurrency mode, boolean prepare) throws Exception {
@@ -171,7 +171,7 @@ public class GridCachePartitionedTxSalvageSelfTest extends GridCommonAbstractTes
      *
      * @param mode Transaction mode (PESSIMISTIC, OPTIMISTIC).
      * @param prepare Whether to prepare transaction state
-     *                (i.e. call {@link GridNearTxLocal#prepare()}).
+     *                (i.e. call {@link GridNearTxLocal#prepare(boolean)}).
      * @throws Exception If failed.
      */
     private void checkSalvageBeforeTimeout(TransactionConcurrency mode, boolean prepare) throws Exception {
@@ -197,7 +197,7 @@ public class GridCachePartitionedTxSalvageSelfTest extends GridCommonAbstractTes
      * Start new transaction on the grid(0) and put some keys to it.
      *
      * @param mode Transaction mode (PESSIMISTIC, OPTIMISTIC).
-     * @param prepare Whether to prepare transaction state (i.e. call {@link GridNearTxLocal#prepare()}).
+     * @param prepare Whether to prepare transaction state (i.e. call {@link GridNearTxLocal#prepare(boolean)}).
      * @throws Exception If failed.
      */
     private void startTxAndPutKeys(final TransactionConcurrency mode, final boolean prepare) throws Exception {
@@ -216,7 +216,7 @@ public class GridCachePartitionedTxSalvageSelfTest extends GridCommonAbstractTes
                         c.put(key, "val" + key);
 
                     if (prepare)
-                        ((TransactionProxyImpl)tx).tx().prepare();
+                        ((TransactionProxyImpl)tx).tx().prepare(true);
                 }
                 catch (IgniteCheckedException e) {
                     info("Failed to put keys to cache: " + e.getMessage());

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaResource.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaResource.java b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaResource.java
index 649f7c4..c63dafa 100644
--- a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaResource.java
+++ b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaResource.java
@@ -56,6 +56,9 @@ final class CacheJtaResource implements XAResource, Synchronization {
     /** */
     private Xid xid;
 
+    /** */
+    private final GridKernalContext ctx;
+
     /**
      * @param cacheTx Cache jta.
      * @param ctx Kernal context.
@@ -65,6 +68,7 @@ final class CacheJtaResource implements XAResource, Synchronization {
         assert ctx != null;
 
         this.cacheTx = cacheTx;
+        this.ctx = ctx;
 
         if (log == null)
             log = U.logger(ctx, logRef, CacheJtaResource.class);
@@ -100,7 +104,7 @@ final class CacheJtaResource implements XAResource, Synchronization {
             log.debug("XA resource rollback(...) [xid=" + xid + "]");
 
         try {
-            cacheTx.rollback();
+            ctx.cache().context().rollbackTxAsync(cacheTx).get();
         }
         catch (IgniteCheckedException e) {
             throwException("Failed to rollback cache transaction: " + e.getMessage(), e);
@@ -118,7 +122,7 @@ final class CacheJtaResource implements XAResource, Synchronization {
             throw new XAException("Cache transaction is not in active state.");
 
         try {
-            cacheTx.prepare();
+            cacheTx.prepare(true);
         }
         catch (IgniteCheckedException e) {
             throwException("Failed to prepare cache transaction.", e);
@@ -146,7 +150,7 @@ final class CacheJtaResource implements XAResource, Synchronization {
             log.debug("XA resource commit(...) [xid=" + xid + ", onePhase=" + onePhase + "]");
 
         try {
-            cacheTx.commit();
+            ctx.cache().context().commitTxAsync(cacheTx).get();
         }
         catch (IgniteCheckedException e) {
             throwException("Failed to commit cache transaction: " + e.getMessage(), e);
@@ -161,9 +165,7 @@ final class CacheJtaResource implements XAResource, Synchronization {
             log.debug("XA resource forget(...) [xid=" + xid + "]");
 
         try {
-            cacheTx.invalidate(true);
-
-            cacheTx.commit();
+            ctx.cache().context().rollbackTxAsync(cacheTx).get();
         }
         catch (IgniteCheckedException e) {
             throwException("Failed to forget cache transaction: " + e.getMessage(), e);
@@ -246,7 +248,7 @@ final class CacheJtaResource implements XAResource, Synchronization {
             throw new CacheException("Cache transaction is not in active state.");
 
         try {
-            cacheTx.prepare();
+            cacheTx.prepare(true);
         }
         catch (IgniteCheckedException e) {
             throw new CacheException("Failed to prepare cache transaction.", e);
@@ -261,7 +263,7 @@ final class CacheJtaResource implements XAResource, Synchronization {
                     log.debug("Synchronization.afterCompletion(STATUS_COMMITTED) [xid=" + cacheTx.xid() + "]");
 
                 try {
-                    cacheTx.commit();
+                    ctx.cache().context().commitTxAsync(cacheTx).get();
                 }
                 catch (IgniteCheckedException e) {
                     throw new CacheException("Failed to commit cache transaction.", e);
@@ -274,7 +276,7 @@ final class CacheJtaResource implements XAResource, Synchronization {
                     log.debug("Synchronization.afterCompletion(STATUS_ROLLEDBACK) [xid=" + cacheTx.xid() + "]");
 
                 try {
-                    cacheTx.rollback();
+                    ctx.cache().context().rollbackTxAsync(cacheTx).get();
                 }
                 catch (IgniteCheckedException e) {
                     throw new CacheException("Failed to rollback cache transaction.", e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/AbstractCacheJtaSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/AbstractCacheJtaSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/AbstractCacheJtaSelfTest.java
deleted file mode 100644
index 37af32c..0000000
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/AbstractCacheJtaSelfTest.java
+++ /dev/null
@@ -1,183 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache;
-
-import javax.transaction.Status;
-import javax.transaction.UserTransaction;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.transactions.Transaction;
-import org.objectweb.jotm.Jotm;
-
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-import static org.apache.ignite.transactions.TransactionState.ACTIVE;
-
-/**
- * Abstract class for cache tests.
- */
-public abstract class AbstractCacheJtaSelfTest extends GridCacheAbstractSelfTest {
-    /** */
-    private static final int GRID_CNT = 1;
-
-    /** Java Open Transaction Manager facade. */
-    protected static Jotm jotm;
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        jotm = new Jotm(true, false);
-
-        super.beforeTestsStarted();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        super.afterTestsStopped();
-
-        jotm.stop();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return GRID_CNT;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        configureJta(cfg);
-
-        CacheConfiguration cfg1 = cacheConfiguration(igniteInstanceName);
-
-        CacheConfiguration cfg2 = cacheConfiguration(igniteInstanceName);
-
-        cfg2.setName("cache-2");
-
-        cfg.setCacheConfiguration(cfg1, cfg2);
-
-        return cfg;
-    }
-
-    /**
-     * @param cfg Ignite Configuration.
-     */
-    protected abstract void configureJta(IgniteConfiguration cfg);
-
-    /**
-     * JUnit.
-     *
-     * @throws Exception If failed.
-     */
-    public void testJta() throws Exception {
-        UserTransaction jtaTx = jotm.getUserTransaction();
-
-        IgniteCache<String, Integer> cache = jcache();
-
-        assert ignite(0).transactions().tx() == null;
-
-        jtaTx.begin();
-
-        try {
-            assert ignite(0).transactions().tx() == null;
-
-            assert cache.getAndPut("key", 1) == null;
-
-            Transaction tx = ignite(0).transactions().tx();
-
-            assert tx != null;
-            assert tx.state() == ACTIVE;
-
-            Integer one = 1;
-
-            assertEquals(one, cache.get("key"));
-
-            tx = ignite(0).transactions().tx();
-
-            assert tx != null;
-            assert tx.state() == ACTIVE;
-
-            jtaTx.commit();
-
-            assert ignite(0).transactions().tx() == null;
-        }
-        finally {
-            if (jtaTx.getStatus() == Status.STATUS_ACTIVE)
-                jtaTx.rollback();
-        }
-
-        assertEquals((Integer)1, cache.get("key"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("ConstantConditions")
-    public void testJtaTwoCaches() throws Exception {
-        UserTransaction jtaTx = jotm.getUserTransaction();
-
-        IgniteEx ignite = grid(0);
-
-        IgniteCache<String, Integer> cache1 = jcache();
-
-        IgniteCache<Object, Object> cache2 = ignite.cache("cache-2");
-
-        assertNull(ignite.transactions().tx());
-
-        jtaTx.begin();
-
-        try {
-            cache1.put("key", 0);
-            cache2.put("key", 0);
-            cache1.put("key1", 1);
-            cache2.put("key2", 2);
-
-            assertEquals(0, (int)cache1.get("key"));
-            assertEquals(0, (int)cache1.get("key"));
-            assertEquals(1, (int)cache1.get("key1"));
-            assertEquals(2, (int)cache2.get("key2"));
-
-            assertEquals(ignite.transactions().tx().state(), ACTIVE);
-
-            jtaTx.commit();
-
-            assertNull(ignite.transactions().tx());
-
-            assertEquals(0, (int)cache1.get("key"));
-            assertEquals(0, (int)cache2.get("key"));
-            assertEquals(1, (int)cache1.get("key1"));
-            assertEquals(2, (int)cache2.get("key2"));
-        }
-        finally {
-            if (jtaTx.getStatus() == Status.STATUS_ACTIVE)
-                jtaTx.rollback();
-        }
-
-        assertEquals(0, (int)cache1.get("key"));
-        assertEquals(0, (int)cache2.get("key"));
-        assertEquals(1, (int)cache1.get("key1"));
-        assertEquals(2, (int)cache2.get("key2"));
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaFactorySelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaFactorySelfTest.java
deleted file mode 100644
index f079974..0000000
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaFactorySelfTest.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache;
-
-import javax.cache.configuration.Factory;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.TransactionConfiguration;
-import org.objectweb.transaction.jta.TransactionManager;
-
-/**
- * Factory JTA integration test using PARTITIONED cache.
- */
-public class GridPartitionedCacheJtaFactorySelfTest extends AbstractCacheJtaSelfTest {
-    /** {@inheritDoc} */
-    @Override protected void configureJta(IgniteConfiguration cfg) {
-        TransactionConfiguration txCfg = cfg.getTransactionConfiguration();
-
-        txCfg.setTxManagerFactory(new Factory<TransactionManager>() {
-            private static final long serialVersionUID = 0L;
-
-            @Override public TransactionManager create() {
-                return jotm.getTransactionManager();
-            }
-        });
-    }
-}

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaLookupClassNameSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaLookupClassNameSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaLookupClassNameSelfTest.java
deleted file mode 100644
index bb1e89c..0000000
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaLookupClassNameSelfTest.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache;
-
-import java.util.concurrent.Callable;
-import javax.transaction.TransactionManager;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.jta.CacheTmLookup;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testsuites.IgniteIgnore;
-
-/**
- * Lookup class name based JTA integration test using PARTITIONED cache.
- */
-public class GridPartitionedCacheJtaLookupClassNameSelfTest extends AbstractCacheJtaSelfTest {
-    /** {@inheritDoc} */
-    @Override protected void configureJta(IgniteConfiguration cfg) {
-        cfg.getTransactionConfiguration().setTxManagerLookupClassName(TestTmLookup.class.getName());
-    }
-
-    /**
-     *
-     */
-    @IgniteIgnore(value = "https://issues.apache.org/jira/browse/IGNITE-1094", forceFailure = true)
-    public void testUncompatibleTmLookup() {
-        final IgniteEx ignite = grid(0);
-
-        final CacheConfiguration cacheCfg = new CacheConfiguration(DEFAULT_CACHE_NAME);
-
-        cacheCfg.setName("Foo");
-        cacheCfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        cacheCfg.setTransactionManagerLookupClassName(TestTmLookup2.class.getName());
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws IgniteException {
-                ignite.createCache(cacheCfg);
-
-                return null;
-            }
-        }, IgniteException.class, null);
-    }
-
-    /**
-     *
-     */
-    @SuppressWarnings("PublicInnerClass")
-    public static class TestTmLookup implements CacheTmLookup {
-        /** {@inheritDoc} */
-        @Override public TransactionManager getTm() {
-            return jotm.getTransactionManager();
-        }
-    }
-
-    /**
-     *
-     */
-    @SuppressWarnings("PublicInnerClass")
-    public static class TestTmLookup2 implements CacheTmLookup {
-        /** {@inheritDoc} */
-        @Override public TransactionManager getTm() {
-            return null;
-        }
-    }
-}

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridReplicatedCacheJtaLookupClassNameSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridReplicatedCacheJtaLookupClassNameSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridReplicatedCacheJtaLookupClassNameSelfTest.java
deleted file mode 100644
index 2b89ba1..0000000
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridReplicatedCacheJtaLookupClassNameSelfTest.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache;
-
-import org.apache.ignite.cache.CacheMode;
-
-import static org.apache.ignite.cache.CacheMode.REPLICATED;
-
-/**
- * Lookup class name based JTA integration test using REPLICATED cache.
- */
-public class GridReplicatedCacheJtaLookupClassNameSelfTest extends GridPartitionedCacheJtaLookupClassNameSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return REPLICATED;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/AbstractCacheJtaSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/AbstractCacheJtaSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/AbstractCacheJtaSelfTest.java
new file mode 100644
index 0000000..89fb72f
--- /dev/null
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/AbstractCacheJtaSelfTest.java
@@ -0,0 +1,248 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.jta;
+
+import javax.transaction.Status;
+import javax.transaction.UserTransaction;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
+import org.apache.ignite.testframework.GridTestSafeThreadFactory;
+import org.apache.ignite.transactions.Transaction;
+import org.objectweb.jotm.Jotm;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED;
+import static org.apache.ignite.transactions.TransactionState.ACTIVE;
+
+/**
+ * Abstract class for cache tests.
+ */
+public abstract class AbstractCacheJtaSelfTest extends GridCacheAbstractSelfTest {
+    /** */
+    private static final int GRID_CNT = 1;
+
+    /** Java Open Transaction Manager facade. */
+    protected static Jotm jotm;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        jotm = new Jotm(true, false);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        jotm.stop();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return GRID_CNT;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        configureJta(cfg);
+
+        CacheConfiguration cfg1 = cacheConfiguration(igniteInstanceName);
+
+        CacheConfiguration cfg2 = cacheConfiguration(igniteInstanceName);
+
+        cfg2.setName("cache-2");
+
+        cfg.setCacheConfiguration(cfg1, cfg2);
+
+        return cfg;
+    }
+
+    /**
+     * @param cfg Ignite Configuration.
+     */
+    protected abstract void configureJta(IgniteConfiguration cfg);
+
+    /**
+     * JUnit.
+     *
+     * @throws Exception If failed.
+     */
+    public void testJta() throws Exception {
+        UserTransaction jtaTx = jotm.getUserTransaction();
+
+        IgniteCache<String, Integer> cache = jcache();
+
+        assert ignite(0).transactions().tx() == null;
+
+        jtaTx.begin();
+
+        try {
+            assert ignite(0).transactions().tx() == null;
+
+            assert cache.getAndPut("key", 1) == null;
+
+            Transaction tx = ignite(0).transactions().tx();
+
+            assert tx != null;
+            assert tx.state() == ACTIVE;
+
+            Integer one = 1;
+
+            assertEquals(one, cache.get("key"));
+
+            tx = ignite(0).transactions().tx();
+
+            assert tx != null;
+            assert tx.state() == ACTIVE;
+
+            jtaTx.commit();
+
+            assert ignite(0).transactions().tx() == null;
+        }
+        finally {
+            if (jtaTx.getStatus() == Status.STATUS_ACTIVE)
+                jtaTx.rollback();
+        }
+
+        assertEquals((Integer)1, cache.get("key"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("ConstantConditions")
+    public void testJtaTwoCaches() throws Exception {
+        UserTransaction jtaTx = jotm.getUserTransaction();
+
+        IgniteEx ignite = grid(0);
+
+        IgniteCache<String, Integer> cache1 = jcache();
+
+        IgniteCache<Object, Object> cache2 = ignite.cache("cache-2");
+
+        assertNull(ignite.transactions().tx());
+
+        jtaTx.begin();
+
+        try {
+            cache1.put("key", 0);
+            cache2.put("key", 0);
+            cache1.put("key1", 1);
+            cache2.put("key2", 2);
+
+            assertEquals(0, (int)cache1.get("key"));
+            assertEquals(0, (int)cache2.get("key"));
+            assertEquals(1, (int)cache1.get("key1"));
+            assertEquals(2, (int)cache2.get("key2"));
+
+            assertEquals(ignite.transactions().tx().state(), ACTIVE);
+
+            jtaTx.commit();
+
+            assertNull(ignite.transactions().tx());
+
+            assertEquals(0, (int)cache1.get("key"));
+            assertEquals(0, (int)cache2.get("key"));
+            assertEquals(1, (int)cache1.get("key1"));
+            assertEquals(2, (int)cache2.get("key2"));
+        }
+        finally {
+            if (jtaTx.getStatus() == Status.STATUS_ACTIVE)
+                jtaTx.rollback();
+        }
+
+        assertEquals(0, (int)cache1.get("key"));
+        assertEquals(0, (int)cache2.get("key"));
+        assertEquals(1, (int)cache1.get("key1"));
+        assertEquals(2, (int)cache2.get("key2"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAsyncOpAwait() throws Exception {
+        final IgniteCache<String, Integer> cache = jcache();
+
+        GridTestSafeThreadFactory factory = new GridTestSafeThreadFactory("JtaThread");
+
+        final CountDownLatch latch = new CountDownLatch(1);
+
+        Callable<Object> c = new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                assertNull(grid(0).transactions().tx());
+
+                UserTransaction jtaTx = jotm.getUserTransaction();
+
+                jtaTx.begin();
+
+                try {
+                    cache.put("key1", 1);
+
+                    cache.putAsync("key", 1);
+
+                    assertEquals(grid(0).transactions().tx().state(), ACTIVE);
+
+                    latch.countDown();
+
+                    info("Before JTA commit.");
+                }
+                finally {
+                    jtaTx.commit();
+                }
+
+                info("After JTA commit.");
+
+                assertEquals((Integer)1, cache.get("key"));
+
+                return null;
+            }
+        };
+
+        Thread task = factory.newThread(c);
+
+        try (Transaction tx = ignite(0).transactions().txStart(PESSIMISTIC, READ_COMMITTED)) {
+            cache.put("key", 0);
+
+            task.start();
+
+            latch.await();
+
+            while (task.getState() != Thread.State.WAITING)
+                factory.checkError();
+
+            info("Before cache TX commit.");
+
+            tx.commit();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactorySelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactorySelfTest.java
new file mode 100644
index 0000000..f6fd5c7
--- /dev/null
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactorySelfTest.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.jta;
+
+import javax.cache.configuration.Factory;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.TransactionConfiguration;
+import org.objectweb.transaction.jta.TransactionManager;
+
+/**
+ * Factory JTA integration test using PARTITIONED cache.
+ */
+public class GridPartitionedCacheJtaFactorySelfTest extends AbstractCacheJtaSelfTest {
+    /** {@inheritDoc} */
+    @Override protected void configureJta(IgniteConfiguration cfg) {
+        TransactionConfiguration txCfg = cfg.getTransactionConfiguration();
+
+        txCfg.setTxManagerFactory(new Factory<TransactionManager>() {
+            private static final long serialVersionUID = 0L;
+
+            @Override public TransactionManager create() {
+                return jotm.getTransactionManager();
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactoryUseSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactoryUseSyncSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactoryUseSyncSelfTest.java
new file mode 100644
index 0000000..b66452d
--- /dev/null
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactoryUseSyncSelfTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.jta;
+
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ * Factory JTA integration test using PARTITIONED cache.
+ */
+public class GridPartitionedCacheJtaFactoryUseSyncSelfTest extends GridPartitionedCacheJtaFactorySelfTest {
+    /** {@inheritDoc} */
+    @Override protected void configureJta(IgniteConfiguration cfg) {
+        super.configureJta(cfg);
+
+        cfg.getTransactionConfiguration().setUseJtaSynchronization(true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaLookupClassNameSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaLookupClassNameSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaLookupClassNameSelfTest.java
new file mode 100644
index 0000000..7357f8e
--- /dev/null
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaLookupClassNameSelfTest.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.jta;
+
+import java.util.concurrent.Callable;
+import javax.transaction.TransactionManager;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.jta.CacheTmLookup;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testsuites.IgniteIgnore;
+
+/**
+ * Lookup class name based JTA integration test using PARTITIONED cache.
+ */
+public class GridPartitionedCacheJtaLookupClassNameSelfTest extends AbstractCacheJtaSelfTest {
+    /** {@inheritDoc} */
+    @Override protected void configureJta(IgniteConfiguration cfg) {
+        cfg.getTransactionConfiguration().setTxManagerLookupClassName(TestTmLookup.class.getName());
+    }
+
+    /**
+     *
+     */
+    @IgniteIgnore(value = "https://issues.apache.org/jira/browse/IGNITE-1094", forceFailure = true)
+    public void testIncompatibleTmLookup() {
+        final IgniteEx ignite = grid(0);
+
+        final CacheConfiguration cacheCfg = new CacheConfiguration(DEFAULT_CACHE_NAME);
+
+        cacheCfg.setName("Foo");
+        cacheCfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        cacheCfg.setTransactionManagerLookupClassName(TestTmLookup2.class.getName());
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws IgniteException {
+                ignite.createCache(cacheCfg);
+
+                return null;
+            }
+        }, IgniteException.class, null);
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings("PublicInnerClass")
+    public static class TestTmLookup implements CacheTmLookup {
+        /** {@inheritDoc} */
+        @Override public TransactionManager getTm() {
+            return jotm.getTransactionManager();
+        }
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings("PublicInnerClass")
+    public static class TestTmLookup2 implements CacheTmLookup {
+        /** {@inheritDoc} */
+        @Override public TransactionManager getTm() {
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridReplicatedCacheJtaFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridReplicatedCacheJtaFactorySelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridReplicatedCacheJtaFactorySelfTest.java
new file mode 100644
index 0000000..1efd99c
--- /dev/null
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridReplicatedCacheJtaFactorySelfTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.jta;
+
+import org.apache.ignite.cache.CacheMode;
+
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+
+/**
+ * Factory JTA integration test using REPLICATED cache.
+ */
+public class GridReplicatedCacheJtaFactorySelfTest extends GridPartitionedCacheJtaFactorySelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return REPLICATED;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridReplicatedCacheJtaFactoryUseSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridReplicatedCacheJtaFactoryUseSyncSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridReplicatedCacheJtaFactoryUseSyncSelfTest.java
new file mode 100644
index 0000000..54610f9
--- /dev/null
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridReplicatedCacheJtaFactoryUseSyncSelfTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.jta;
+
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ * Factory JTA integration test using REPLICATED cache.
+ */
+public class GridReplicatedCacheJtaFactoryUseSyncSelfTest extends GridReplicatedCacheJtaFactorySelfTest {
+    /** {@inheritDoc} */
+    @Override protected void configureJta(IgniteConfiguration cfg) {
+        super.configureJta(cfg);
+
+        cfg.getTransactionConfiguration().setUseJtaSynchronization(true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridReplicatedCacheJtaLookupClassNameSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridReplicatedCacheJtaLookupClassNameSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridReplicatedCacheJtaLookupClassNameSelfTest.java
new file mode 100644
index 0000000..a1aa6cf
--- /dev/null
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridReplicatedCacheJtaLookupClassNameSelfTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.jta;
+
+import org.apache.ignite.cache.CacheMode;
+
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+
+/**
+ * Lookup class name based JTA integration test using REPLICATED cache.
+ */
+public class GridReplicatedCacheJtaLookupClassNameSelfTest extends GridPartitionedCacheJtaLookupClassNameSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return REPLICATED;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/package-info.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/package-info.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/package-info.java
new file mode 100644
index 0000000..49b208d
--- /dev/null
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * <!-- Package description. -->
+ * Contains internal tests or test related classes and interfaces.
+ */
+package org.apache.ignite.internal.processors.cache.jta;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/jta/src/test/java/org/apache/ignite/testsuites/IgniteJtaTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/testsuites/IgniteJtaTestSuite.java b/modules/jta/src/test/java/org/apache/ignite/testsuites/IgniteJtaTestSuite.java
index 4ae5df0..677f485 100644
--- a/modules/jta/src/test/java/org/apache/ignite/testsuites/IgniteJtaTestSuite.java
+++ b/modules/jta/src/test/java/org/apache/ignite/testsuites/IgniteJtaTestSuite.java
@@ -21,12 +21,12 @@ import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.cache.CacheJndiTmFactorySelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheJtaConfigurationValidationSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheJtaFactoryConfigValidationSelfTest;
-import org.apache.ignite.internal.processors.cache.GridPartitionedCacheJtaFactorySelfTest;
-import org.apache.ignite.internal.processors.cache.GridPartitionedCacheJtaFactoryUseSyncSelfTest;
-import org.apache.ignite.internal.processors.cache.GridPartitionedCacheJtaLookupClassNameSelfTest;
-import org.apache.ignite.internal.processors.cache.GridReplicatedCacheJtaFactorySelfTest;
-import org.apache.ignite.internal.processors.cache.GridReplicatedCacheJtaFactoryUseSyncSelfTest;
-import org.apache.ignite.internal.processors.cache.GridReplicatedCacheJtaLookupClassNameSelfTest;
+import org.apache.ignite.internal.processors.cache.jta.GridPartitionedCacheJtaFactorySelfTest;
+import org.apache.ignite.internal.processors.cache.jta.GridPartitionedCacheJtaFactoryUseSyncSelfTest;
+import org.apache.ignite.internal.processors.cache.jta.GridPartitionedCacheJtaLookupClassNameSelfTest;
+import org.apache.ignite.internal.processors.cache.jta.GridReplicatedCacheJtaFactorySelfTest;
+import org.apache.ignite.internal.processors.cache.jta.GridReplicatedCacheJtaFactoryUseSyncSelfTest;
+import org.apache.ignite.internal.processors.cache.jta.GridReplicatedCacheJtaLookupClassNameSelfTest;
 import org.apache.ignite.internal.processors.cache.GridJtaLifecycleAwareSelfTest;
 import org.apache.ignite.testframework.IgniteTestSuite;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e96225f1/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTransactionalTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTransactionalTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTransactionalTest.cs
index 45d0b5f..77ae8fe 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTransactionalTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTransactionalTest.cs
@@ -606,8 +606,7 @@ namespace Apache.Ignite.Core.Tests.Cache
         /// with multiple participating caches.
         /// </summary>
         [Test]
-        [Ignore("IGNITE-1561")]
-        public void TestTransactionScopeMultiCache()
+        public void TestTransactionScopeMultiCache([Values(true, false)] bool async)
         {
             var cache1 = Cache();
 
@@ -622,8 +621,16 @@ namespace Apache.Ignite.Core.Tests.Cache
             // Commit.
             using (var ts = new TransactionScope())
             {
-                cache1[1] = 10;
-                cache2[1] = 20;
+                if (async)
+                {
+                    cache1.PutAsync(1, 10);
+                    cache2.PutAsync(1, 20);
+                }
+                else
+                {
+                    cache1.Put(1, 10);
+                    cache2.Put(1, 20);
+                }
 
                 ts.Complete();
             }
@@ -634,8 +641,16 @@ namespace Apache.Ignite.Core.Tests.Cache
             // Rollback.
             using (new TransactionScope())
             {
-                cache1[1] = 100;
-                cache2[1] = 200;
+                if (async)
+                {
+                    cache1.PutAsync(1, 100);
+                    cache2.PutAsync(1, 200);
+                }
+                else
+                {
+                    cache1.Put(1, 100);
+                    cache2.Put(1, 200);
+                }
             }
 
             Assert.AreEqual(10, cache1[1]);
@@ -773,10 +788,10 @@ namespace Apache.Ignite.Core.Tests.Cache
             for (var i = 0; i < 10; i++)
             {
                 CheckTxOp((cache, key) => cache.Put(key, -5));
-                CheckTxOp((cache, key) => cache.PutAsync(key, -5).Wait());
+                CheckTxOp((cache, key) => cache.PutAsync(key, -5));
 
                 CheckTxOp((cache, key) => cache.PutAll(new Dictionary<int, int> {{key, -7}}));
-                CheckTxOp((cache, key) => cache.PutAllAsync(new Dictionary<int, int> {{key, -7}}).Wait());
+                CheckTxOp((cache, key) => cache.PutAllAsync(new Dictionary<int, int> {{key, -7}}));
 
                 CheckTxOp((cache, key) =>
                 {
@@ -786,11 +801,11 @@ namespace Apache.Ignite.Core.Tests.Cache
                 CheckTxOp((cache, key) =>
                 {
                     cache.Remove(key);
-                    cache.PutIfAbsentAsync(key, -10).Wait();
+                    cache.PutIfAbsentAsync(key, -10);
                 });
 
                 CheckTxOp((cache, key) => cache.GetAndPut(key, -9));
-                CheckTxOp((cache, key) => cache.GetAndPutAsync(key, -9).Wait());
+                CheckTxOp((cache, key) => cache.GetAndPutAsync(key, -9));
 
                 CheckTxOp((cache, key) =>
                 {
@@ -800,32 +815,32 @@ namespace Apache.Ignite.Core.Tests.Cache
                 CheckTxOp((cache, key) =>
                 {
                     cache.Remove(key);
-                    cache.GetAndPutIfAbsentAsync(key, -10).Wait();
+                    cache.GetAndPutIfAbsentAsync(key, -10);
                 });
 
                 CheckTxOp((cache, key) => cache.GetAndRemove(key));
-                CheckTxOp((cache, key) => cache.GetAndRemoveAsync(key).Wait());
+                CheckTxOp((cache, key) => cache.GetAndRemoveAsync(key));
 
                 CheckTxOp((cache, key) => cache.GetAndReplace(key, -11));
-                CheckTxOp((cache, key) => cache.GetAndReplaceAsync(key, -11).Wait());
+                CheckTxOp((cache, key) => cache.GetAndReplaceAsync(key, -11));
 
                 CheckTxOp((cache, key) => cache.Invoke(key, new AddProcessor(), 1));
-                CheckTxOp((cache, key) => cache.InvokeAsync(key, new AddProcessor(), 1).Wait());
+                CheckTxOp((cache, key) => cache.InvokeAsync(key, new AddProcessor(), 1));
 
                 CheckTxOp((cache, key) => cache.InvokeAll(new[] {key}, new AddProcessor(), 1));
-                CheckTxOp((cache, key) => cache.InvokeAllAsync(new[] {key}, new AddProcessor(), 1).Wait());
+                CheckTxOp((cache, key) => cache.InvokeAllAsync(new[] {key}, new AddProcessor(), 1));
 
                 CheckTxOp((cache, key) => cache.Remove(key));
-                CheckTxOp((cache, key) => cache.RemoveAsync(key).Wait());
+                CheckTxOp((cache, key) => cache.RemoveAsync(key));
 
                 CheckTxOp((cache, key) => cache.RemoveAll(new[] {key}));
-                CheckTxOp((cache, key) => cache.RemoveAllAsync(new[] {key}).Wait());
+                CheckTxOp((cache, key) => cache.RemoveAllAsync(new[] {key}));
 
                 CheckTxOp((cache, key) => cache.Replace(key, 100));
-                CheckTxOp((cache, key) => cache.ReplaceAsync(key, 100).Wait());
+                CheckTxOp((cache, key) => cache.ReplaceAsync(key, 100));
 
                 CheckTxOp((cache, key) => cache.Replace(key, cache[key], 100));
-                CheckTxOp((cache, key) => cache.ReplaceAsync(key, cache[key], 100).Wait());
+                CheckTxOp((cache, key) => cache.ReplaceAsync(key, cache[key], 100));
             }
         }