You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/02/02 15:21:57 UTC

[1/4] incubator-ignite git commit: # ignite-26 renamed IgniteTxEx - IgniteInternalTx

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-26 cd4040d5b -> d89ef5bdf


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
index d1b6ce6..16a22e6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
@@ -402,7 +402,7 @@ public class GridCacheTestEntryEx<K, V> extends GridMetadataAwareAdapter impleme
     }
 
     /** @inheritDoc */
-    @Override public V innerGet(@Nullable IgniteTxEx<K, V> tx,
+    @Override public V innerGet(@Nullable IgniteInternalTx<K, V> tx,
         boolean readSwap,
         boolean readThrough,
         boolean failFast,
@@ -424,7 +424,7 @@ public class GridCacheTestEntryEx<K, V> extends GridMetadataAwareAdapter impleme
     }
 
     /** @inheritDoc */
-    @Override public GridCacheUpdateTxResult<V> innerSet(@Nullable IgniteTxEx<K, V> tx, UUID evtNodeId, UUID affNodeId,
+    @Override public GridCacheUpdateTxResult<V> innerSet(@Nullable IgniteInternalTx<K, V> tx, UUID evtNodeId, UUID affNodeId,
         @Nullable V val, @Nullable byte[] valBytes, boolean writeThrough, boolean retval, long ttl,
         boolean evt, boolean metrics, long topVer, IgnitePredicate<CacheEntry<K, V>>[] filter, GridDrType drType,
         long drExpireTime, @Nullable GridCacheVersion drVer, UUID subjId, String taskName) throws IgniteCheckedException,
@@ -488,7 +488,7 @@ public class GridCacheTestEntryEx<K, V> extends GridMetadataAwareAdapter impleme
     }
 
     /** @inheritDoc */
-    @Override public GridCacheUpdateTxResult<V> innerRemove(@Nullable IgniteTxEx<K, V> tx, UUID evtNodeId,
+    @Override public GridCacheUpdateTxResult<V> innerRemove(@Nullable IgniteInternalTx<K, V> tx, UUID evtNodeId,
         UUID affNodeId, boolean writeThrough, boolean retval, boolean evt, boolean metrics, long topVer,
         IgnitePredicate<CacheEntry<K, V>>[] filter, GridDrType drType, @Nullable GridCacheVersion drVer, UUID subjId,
         String taskName)
@@ -515,12 +515,12 @@ public class GridCacheTestEntryEx<K, V> extends GridMetadataAwareAdapter impleme
     }
 
     /** @inheritDoc */
-    @Override public boolean tmLock(IgniteTxEx<K, V> tx, long timeout) {
+    @Override public boolean tmLock(IgniteInternalTx<K, V> tx, long timeout) {
         assert false; return false;
     }
 
     /** @inheritDoc */
-    @Override public void txUnlock(IgniteTxEx<K, V> tx) {
+    @Override public void txUnlock(IgniteInternalTx<K, V> tx) {
         assert false;
     }
 
@@ -592,7 +592,7 @@ public class GridCacheTestEntryEx<K, V> extends GridMetadataAwareAdapter impleme
 
     /** @inheritDoc */
     @Override public GridTuple<V> peek0(boolean failFast, GridCachePeekMode mode,
-        IgnitePredicate<CacheEntry<K, V>>[] filter, IgniteTxEx<K, V> tx)
+        IgnitePredicate<CacheEntry<K, V>>[] filter, IgniteInternalTx<K, V> tx)
         throws GridCacheEntryRemovedException, GridCacheFilterFailedException, IgniteCheckedException {
         return F.t(val);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestStore.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestStore.java
index c8d20e7..d6784b9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestStore.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestStore.java
@@ -329,7 +329,7 @@ public final class GridCacheTestStore extends CacheStore<Integer, String> {
 
         assertTrue("Unexpected tx class: " + tx.getClass(), tx instanceof IgniteTxProxy);
 
-        IgniteTxEx tx0 = GridTestUtils.getFieldValue(tx, "tx");
+        IgniteInternalTx tx0 = GridTestUtils.getFieldValue(tx, "tx");
 
         if (!tx0.local())
             throw new IgniteException("Tx is not local: " + tx);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxOriginatingNodeFailureAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxOriginatingNodeFailureAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxOriginatingNodeFailureAbstractSelfTest.java
index 14a5c5c..ca47cef 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxOriginatingNodeFailureAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxOriginatingNodeFailureAbstractSelfTest.java
@@ -157,7 +157,7 @@ public abstract class IgniteTxOriginatingNodeFailureAbstractSelfTest extends Gri
 
                 IgniteTxProxyImpl tx = (IgniteTxProxyImpl)cache.txStart();
 
-                IgniteTxEx txEx = GridTestUtils.getFieldValue(tx, "tx");
+                IgniteInternalTx txEx = GridTestUtils.getFieldValue(tx, "tx");
 
                 cache.putAll(map);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxMultiNodeSelfTest.java
index fe06d27..c1d67ce 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxMultiNodeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxMultiNodeSelfTest.java
@@ -235,12 +235,12 @@ public class GridCacheNearTxMultiNodeSelfTest extends GridCommonAbstractTest {
      */
     @SuppressWarnings( {"unchecked"})
     private void checkTm(Ignite g, IgniteTxManager tm) {
-        Collection<IgniteTxEx> txs = tm.txs();
+        Collection<IgniteInternalTx> txs = tm.txs();
 
         info(">>> Number of transactions in the set [size=" + txs.size() +
             ", nodeId=" + g.cluster().localNode().id() + ']');
 
-        for (IgniteTxEx tx : txs)
+        for (IgniteInternalTx tx : txs)
             assert tx.done() : "Transaction is not finished: " + tx;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/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 590568e..a5c384f 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
@@ -141,7 +141,7 @@ public class GridCachePartitionedTxSalvageSelfTest extends GridCommonAbstractTes
      *
      * @param mode Transaction mode (PESSIMISTIC, OPTIMISTIC).
      * @param prepare Whether to preapre transaction state
-     *                (i.e. call {@link IgniteTxEx#prepare()}).
+     *                (i.e. call {@link org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx#prepare()}).
      * @throws Exception If failed.
      */
     private void checkSalvageAfterTimeout(IgniteTxConcurrency mode, boolean prepare) throws Exception {
@@ -160,7 +160,7 @@ public class GridCachePartitionedTxSalvageSelfTest extends GridCommonAbstractTes
      *
      * @param mode Transaction mode (PESSIMISTIC, OPTIMISTIC).
      * @param prepare Whether to preapre transaction state
-     *                (i.e. call {@link IgniteTxEx#prepare()}).
+     *                (i.e. call {@link org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx#prepare()}).
      * @throws Exception If failed.
      */
     private void checkSalvageBeforeTimeout(IgniteTxConcurrency mode, boolean prepare) throws Exception {
@@ -187,7 +187,7 @@ public class GridCachePartitionedTxSalvageSelfTest extends GridCommonAbstractTes
      *
      * @param mode Transaction mode (PESSIMISTIC, OPTIMISTIC).
      * @param prepare Whether to preapre transaction state
-     *                (i.e. call {@link IgniteTxEx#prepare()}).
+     *                (i.e. call {@link org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx#prepare()}).
      * @throws Exception If failed.
      */
     private void startTxAndPutKeys(final IgniteTxConcurrency mode, final boolean prepare) throws Exception {
@@ -207,7 +207,7 @@ public class GridCachePartitionedTxSalvageSelfTest extends GridCommonAbstractTes
 
                     // Unproxy.
                     if (prepare)
-                        U.<IgniteTxEx>field(tx, "tx").prepare();
+                        U.<IgniteInternalTx>field(tx, "tx").prepare();
                 }
                 catch (IgniteCheckedException e) {
                     info("Failed to put keys to cache: " + e.getMessage());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsDataManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsDataManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsDataManagerSelfTest.java
index 92d18ab..544d2f5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsDataManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsDataManagerSelfTest.java
@@ -191,7 +191,7 @@ public class GridGgfsDataManagerSelfTest extends GridGgfsCommonAbstractTest {
             for (int j = 0; j < NODES_CNT; j++) {
                 GridCacheContext<Object, Object> ctx = GridTestUtils.getFieldValue(grid(j).cachex(DATA_CACHE_NAME),
                     "ctx");
-                Collection<IgniteTxEx<Object, Object>> txs = ctx.tm().txs();
+                Collection<IgniteInternalTx<Object, Object>> txs = ctx.tm().txs();
 
                 assert txs.isEmpty() : "Incomplete transactions: " + txs;
             }
@@ -279,7 +279,7 @@ public class GridGgfsDataManagerSelfTest extends GridGgfsCommonAbstractTest {
             for (int j = 0; j < NODES_CNT; j++) {
                 GridCacheContext<Object, Object> ctx = GridTestUtils.getFieldValue(grid(j).cachex(DATA_CACHE_NAME),
                     "ctx");
-                Collection<IgniteTxEx<Object, Object>> txs = ctx.tm().txs();
+                Collection<IgniteInternalTx<Object, Object>> txs = ctx.tm().txs();
 
                 assert txs.isEmpty() : "Incomplete transactions: " + txs;
             }
@@ -359,7 +359,7 @@ public class GridGgfsDataManagerSelfTest extends GridGgfsCommonAbstractTest {
             for (int j = 0; j < NODES_CNT; j++) {
                 GridCacheContext<Object, Object> ctx = GridTestUtils.getFieldValue(grid(j).cachex(DATA_CACHE_NAME),
                     "ctx");
-                Collection<IgniteTxEx<Object, Object>> txs = ctx.tm().txs();
+                Collection<IgniteInternalTx<Object, Object>> txs = ctx.tm().txs();
 
                 assert txs.isEmpty() : "Incomplete transactions: " + txs;
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java
index 8e6efd7..5f7b826 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java
@@ -81,11 +81,11 @@ public class GridHashMapLoadTest extends GridCommonAbstractTest {
 
             map.put(key, new GridCacheMapEntry<Integer, Integer>(ctx, key,
                 key.hashCode(), val, null, 0, 1) {
-                @Override public boolean tmLock(IgniteTxEx<Integer, Integer> tx, long timeout) {
+                @Override public boolean tmLock(IgniteInternalTx<Integer, Integer> tx, long timeout) {
                     return false;
                 }
 
-                @Override public void txUnlock(IgniteTxEx<Integer, Integer> tx) {
+                @Override public void txUnlock(IgniteInternalTx<Integer, Integer> tx) {
                     // No-op.
                 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
index 98f86bd..ddc9a56 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
@@ -1188,7 +1188,7 @@ public class GridHadoopDefaultMapReducePlannerSelfTest extends GridHadoopAbstrac
         }
 
         /** {@inheritDoc} */
-        @Override public ClusterMetrics metrics() throws IgniteCheckedException {
+        @Override public ClusterMetrics metrics() {
             return null;
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
index 52cf253..02ad658 100644
--- a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
+++ b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
@@ -21,7 +21,6 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.jta.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.jetbrains.annotations.*;
 
@@ -73,7 +72,7 @@ public class CacheJtaManager<K, V> extends CacheJtaManagerAdapter<K, V> {
                     Transaction jtaTx = jtaTm.getTransaction();
 
                     if (jtaTx != null) {
-                        IgniteTxEx tx = cctx.tm().userTx();
+                        IgniteInternalTx tx = cctx.tm().userTx();
 
                         if (tx == null) {
                             TransactionsConfiguration tCfg = cctx.kernalContext().config()
@@ -94,7 +93,7 @@ public class CacheJtaManager<K, V> extends CacheJtaManagerAdapter<K, V> {
                             );
                         }
 
-                        rsrc = new GridCacheXAResource((IgniteTxEx)tx, cctx);
+                        rsrc = new GridCacheXAResource((IgniteInternalTx)tx, cctx);
 
                         if (!jtaTx.enlistResource(rsrc))
                             throw new IgniteCheckedException("Failed to enlist XA resource to JTA user transaction.");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java
index 1c15582..f98197f 100644
--- a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java
+++ b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java
@@ -42,7 +42,7 @@ public final class GridCacheXAResource implements XAResource {
     private GridCacheContext cctx;
 
     /** Cache transaction. */
-    private IgniteTxEx cacheTx;
+    private IgniteInternalTx cacheTx;
 
     /** */
     private IgniteLogger log;
@@ -54,7 +54,7 @@ public final class GridCacheXAResource implements XAResource {
      * @param cacheTx Cache jta.
      * @param cctx Cache context.
      */
-    public GridCacheXAResource(IgniteTxEx cacheTx, GridCacheContext cctx) {
+    public GridCacheXAResource(IgniteInternalTx cacheTx, GridCacheContext cctx) {
         assert cacheTx != null;
         assert cctx != null;
 


[4/4] incubator-ignite git commit: # ignite-26 renamed IgniteTxEx - IgniteInternalTx

Posted by sb...@apache.org.
# ignite-26 renamed IgniteTxEx - IgniteInternalTx


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

Branch: refs/heads/ignite-26
Commit: d89ef5bdf6befd8476f457da3367ca1ed9347f17
Parents: cd4040d
Author: sboikov <sb...@gridgain.com>
Authored: Mon Feb 2 17:21:21 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Feb 2 17:21:23 2015 +0300

----------------------------------------------------------------------
 .../ClientAbstractMultiNodeSelfTest.java        |   2 +-
 .../apache/ignite/cache/CacheProjection.java    |   2 +-
 .../org/apache/ignite/cluster/ClusterGroup.java |   4 +-
 .../ignite/internal/ClusterGroupAdapter.java    |   4 +-
 .../ignite/internal/IgniteClusterAsyncImpl.java |   2 +-
 .../ignite/internal/IgniteTransactionsEx.java   |   8 +-
 .../processors/cache/GridCacheAdapter.java      |  22 +-
 .../processors/cache/GridCacheEntryEx.java      |  12 +-
 .../processors/cache/GridCacheEntryImpl.java    |   4 +-
 .../processors/cache/GridCacheEventManager.java |   6 +-
 .../cache/GridCacheEvictionEntry.java           |   4 +-
 .../processors/cache/GridCacheMapEntry.java     |  22 +-
 .../cache/GridCacheMultiTxFuture.java           |  17 +-
 .../processors/cache/GridCacheMvcc.java         |   2 +-
 .../processors/cache/GridCacheMvccManager.java  |   4 +-
 .../cache/GridCacheProjectionImpl.java          |   2 +-
 .../processors/cache/GridCacheProxyImpl.java    |   2 +-
 .../cache/GridCacheSharedContext.java           |  11 +-
 .../processors/cache/GridCacheStoreManager.java |  26 +-
 .../processors/cache/GridCacheUtils.java        |  16 +-
 .../datastructures/GridCacheAtomicLongImpl.java |  17 +-
 .../GridCacheAtomicReferenceImpl.java           |   5 +-
 .../GridCacheAtomicSequenceImpl.java            |   3 +-
 .../GridCacheAtomicStampedImpl.java             |   5 +-
 .../GridCacheCountDownLatchImpl.java            |   4 +-
 .../GridCacheDataStructuresManager.java         |  16 +-
 .../GridTransactionalCacheQueueImpl.java        |   9 +-
 .../distributed/GridCacheCommittedTxInfo.java   |   2 +-
 ...ridCacheOptimisticCheckPreparedTxFuture.java |   4 +-
 ...idCacheOptimisticCheckPreparedTxRequest.java |   2 +-
 .../GridCachePerThreadTxCommitBuffer.java       |   2 +-
 ...dCachePessimisticCheckCommittedTxFuture.java |   4 +-
 ...CachePessimisticCheckCommittedTxRequest.java |   2 +-
 .../distributed/GridCacheTxCommitBuffer.java    |   2 +-
 .../distributed/GridDistributedCacheEntry.java  |   4 +-
 .../GridDistributedTxPrepareRequest.java        |   2 +-
 .../GridDistributedTxRemoteAdapter.java         |  10 +-
 .../distributed/dht/GridDhtCacheEntry.java      |   4 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |   6 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |  12 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |  16 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |  16 +-
 .../distributed/near/GridNearCacheAdapter.java  |   4 +-
 .../distributed/near/GridNearCacheEntry.java    |   4 +-
 .../near/GridNearTxFinishFuture.java            |  16 +-
 .../cache/distributed/near/GridNearTxLocal.java |  52 +-
 .../near/GridNearTxPrepareFuture.java           |  22 +-
 .../near/GridNearTxPrepareRequest.java          |   2 +-
 .../cache/local/GridLocalCacheEntry.java        |   4 +-
 .../processors/cache/local/GridLocalTx.java     |  10 +-
 .../cache/local/GridLocalTxFuture.java          |   4 +-
 .../cache/transactions/IgniteInternalTx.java    | 706 +++++++++++++++++++
 .../transactions/IgniteTransactionsImpl.java    |  18 +-
 .../cache/transactions/IgniteTxAdapter.java     |  24 +-
 .../cache/transactions/IgniteTxEntry.java       |   6 +-
 .../cache/transactions/IgniteTxEx.java          | 706 -------------------
 .../cache/transactions/IgniteTxHandler.java     |  58 +-
 .../transactions/IgniteTxLocalAdapter.java      |  16 +-
 .../cache/transactions/IgniteTxLocalEx.java     |   2 +-
 .../cache/transactions/IgniteTxManager.java     | 168 ++---
 .../cache/transactions/IgniteTxProxyImpl.java   |  14 +-
 .../cache/transactions/IgniteTxRemoteEx.java    |   2 +-
 .../processors/fs/GridGgfsDataManager.java      |   5 +-
 .../processors/fs/GridGgfsMetaManager.java      |  26 +-
 .../handlers/cache/GridCacheCommandHandler.java |   3 +-
 .../service/GridServiceProcessor.java           |   3 +-
 .../processors/cache/GridCacheTestEntryEx.java  |  12 +-
 .../processors/cache/GridCacheTestStore.java    |   2 +-
 ...xOriginatingNodeFailureAbstractSelfTest.java |   2 +-
 .../near/GridCacheNearTxMultiNodeSelfTest.java  |   4 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |   8 +-
 .../fs/GridGgfsDataManagerSelfTest.java         |   6 +-
 .../loadtests/hashmap/GridHashMapLoadTest.java  |   4 +-
 ...idHadoopDefaultMapReducePlannerSelfTest.java |   2 +-
 .../processors/cache/jta/CacheJtaManager.java   |   5 +-
 .../cache/jta/GridCacheXAResource.java          |   4 +-
 76 files changed, 1116 insertions(+), 1127 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/clients/src/test/java/org/apache/ignite/client/integration/ClientAbstractMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/client/integration/ClientAbstractMultiNodeSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/client/integration/ClientAbstractMultiNodeSelfTest.java
index de1a398..2a3ef69 100644
--- a/modules/clients/src/test/java/org/apache/ignite/client/integration/ClientAbstractMultiNodeSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/client/integration/ClientAbstractMultiNodeSelfTest.java
@@ -791,7 +791,7 @@ public abstract class ClientAbstractMultiNodeSelfTest extends GridCommonAbstract
 
             GridCacheVersion v = ((GridCacheVersionable)o).version();
 
-            IgniteTxEx t = tm.tx(v);
+            IgniteInternalTx t = tm.tx(v);
 
             if (t.hasWriteKey(cacheCtx.txKey("x1")))
                 assertFalse("Invalid tx flags: " + t, t.syncCommit());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/cache/CacheProjection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheProjection.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheProjection.java
index 33c6d1e..c3cc619 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheProjection.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheProjection.java
@@ -1171,7 +1171,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      * @param isolation Isolation.
      * @return New transaction.
      */
-    public IgniteTxEx txStartEx(IgniteTxConcurrency concurrency, IgniteTxIsolation isolation);
+    public IgniteInternalTx txStartEx(IgniteTxConcurrency concurrency, IgniteTxIsolation isolation);
 
     /**
      * Starts transaction with specified isolation, concurrency, timeout, invalidation flag,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/cluster/ClusterGroup.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cluster/ClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/cluster/ClusterGroup.java
index 2237d9d..9841fc2 100644
--- a/modules/core/src/main/java/org/apache/ignite/cluster/ClusterGroup.java
+++ b/modules/core/src/main/java/org/apache/ignite/cluster/ClusterGroup.java
@@ -250,7 +250,7 @@ public interface ClusterGroup {
      * Gets a metrics snapshot for this projection.
      *
      * @return Grid projection metrics snapshot.
-     * @throws IgniteCheckedException If projection is empty.
+     * @throws IgniteException If projection is empty.
      */
-    public ClusterMetrics metrics() throws IgniteCheckedException;
+    public ClusterMetrics metrics() throws IgniteException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/ClusterGroupAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/ClusterGroupAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/ClusterGroupAdapter.java
index 528e9a1..38c02bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/ClusterGroupAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/ClusterGroupAdapter.java
@@ -264,12 +264,12 @@ public class ClusterGroupAdapter implements ClusterGroupEx, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Override public final ClusterMetrics metrics() throws IgniteCheckedException {
+    @Override public final ClusterMetrics metrics() {
         guard();
 
         try {
             if (nodes().isEmpty())
-                throw U.emptyTopologyException();
+                throw U.convertException(U.emptyTopologyException());
 
             return new ClusterMetricsSnapshot(this);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/IgniteClusterAsyncImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteClusterAsyncImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteClusterAsyncImpl.java
index c684217..4b966e4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteClusterAsyncImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteClusterAsyncImpl.java
@@ -245,7 +245,7 @@ public class IgniteClusterAsyncImpl extends IgniteAsyncSupportAdapter<IgniteClus
     }
 
     /** {@inheritDoc} */
-    @Override public ClusterMetrics metrics() throws IgniteCheckedException {
+    @Override public ClusterMetrics metrics() {
         return grid.metrics();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/IgniteTransactionsEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteTransactionsEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteTransactionsEx.java
index 6efe99f..08a9fe0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteTransactionsEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteTransactionsEx.java
@@ -49,7 +49,7 @@ public interface IgniteTransactionsEx extends IgniteTransactions {
      * @param txSize Number of entries participating in transaction (may be approximate).
      * @return New transaction.
      */
-    public IgniteTxEx txStartEx(GridCacheContext ctx,
+    public IgniteInternalTx txStartEx(GridCacheContext ctx,
         IgniteTxConcurrency concurrency,
         IgniteTxIsolation isolation,
         long timeout,
@@ -61,7 +61,7 @@ public interface IgniteTransactionsEx extends IgniteTransactions {
      * @param isolation Isolation.
      * @return New transaction.
      */
-    public IgniteTxEx txStartEx(GridCacheContext ctx, IgniteTxConcurrency concurrency, IgniteTxIsolation isolation);
+    public IgniteInternalTx txStartEx(GridCacheContext ctx, IgniteTxConcurrency concurrency, IgniteTxIsolation isolation);
 
     /**
      * @param ctx Cache context.
@@ -73,7 +73,7 @@ public interface IgniteTransactionsEx extends IgniteTransactions {
      * @return New transaction.
      * @throws IgniteCheckedException If failed.
      */
-    public IgniteTxEx txStartPartitionEx(GridCacheContext ctx,
+    public IgniteInternalTx txStartPartitionEx(GridCacheContext ctx,
         int partId,
         IgniteTxConcurrency concurrency,
         IgniteTxIsolation isolation,
@@ -90,7 +90,7 @@ public interface IgniteTransactionsEx extends IgniteTransactions {
      * @return New transaction.
      * @throws IgniteCheckedException If failed.
      */
-    public IgniteTxEx txStartAffinity(GridCacheContext ctx,
+    public IgniteInternalTx txStartAffinity(GridCacheContext ctx,
         Object affinityKey,
         IgniteTxConcurrency concurrency,
         IgniteTxIsolation isolation,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/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 3f9753c..9e06d86 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
@@ -720,7 +720,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
                 }
             }
 
-            IgniteTxEx<K, V> tx = ctx.tm().localTx();
+            IgniteInternalTx<K, V> tx = ctx.tm().localTx();
 
             if (tx != null) {
                 GridTuple<V> peek = tx.peek(ctx, failFast, key, filter);
@@ -799,7 +799,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      * @return Peeked value.
      * @throws IgniteCheckedException In case of error.
      */
-    @Nullable protected V peek0(K key, @Nullable Collection<GridCachePeekMode> modes, IgniteTxEx<K, V> tx)
+    @Nullable protected V peek0(K key, @Nullable Collection<GridCachePeekMode> modes, IgniteInternalTx<K, V> tx)
         throws IgniteCheckedException {
         try {
             GridTuple<V> peek = peek0(false, key, modes, tx);
@@ -825,7 +825,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      * @throws GridCacheFilterFailedException If filer validation failed.
      */
     @Nullable protected GridTuple<V> peek0(boolean failFast, K key, @Nullable Collection<GridCachePeekMode> modes,
-        IgniteTxEx<K, V> tx) throws IgniteCheckedException, GridCacheFilterFailedException {
+        IgniteInternalTx<K, V> tx) throws IgniteCheckedException, GridCacheFilterFailedException {
         if (F.isEmpty(modes))
             return F.t(peek(key, (IgnitePredicate<CacheEntry<K, V>>)null));
 
@@ -896,7 +896,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      * @throws IgniteCheckedException If failed.
      */
     protected Map<K, V> peekAll0(@Nullable Collection<? extends K> keys, @Nullable Collection<GridCachePeekMode> modes,
-        IgniteTxEx<K, V> tx, @Nullable Collection<K> skipped) throws IgniteCheckedException {
+        IgniteInternalTx<K, V> tx, @Nullable Collection<K> skipped) throws IgniteCheckedException {
         if (F.isEmpty(keys))
             return emptyMap();
 
@@ -1467,7 +1467,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      */
     public IgniteInternalFuture<Object> readThroughAllAsync(final Collection<? extends K> keys,
         boolean reload,
-        @Nullable final IgniteTxEx<K, V> tx,
+        @Nullable final IgniteInternalTx<K, V> tx,
         IgnitePredicate<CacheEntry<K, V>>[] filter,
         @Nullable UUID subjId,
         String taskName,
@@ -3561,7 +3561,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteTxEx txStartEx(IgniteTxConcurrency concurrency, IgniteTxIsolation isolation) {
+    @Override public IgniteInternalTx txStartEx(IgniteTxConcurrency concurrency, IgniteTxIsolation isolation) {
         IgniteTransactionsEx txs = ctx.kernalContext().cache().transactions();
 
         return txs.txStartEx(ctx, concurrency, isolation);
@@ -4049,7 +4049,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      * @return Transaction commit future.
      */
     @SuppressWarnings("unchecked")
-    public IgniteInternalFuture<IgniteTxEx> commitTxAsync(final IgniteTxEx tx) {
+    public IgniteInternalFuture<IgniteInternalTx> commitTxAsync(final IgniteInternalTx tx) {
         FutureHolder holder = lastFut.get();
 
         holder.lock();
@@ -4058,9 +4058,9 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
             IgniteInternalFuture fut = holder.future();
 
             if (fut != null && !fut.isDone()) {
-                IgniteInternalFuture<IgniteTxEx> f = new GridEmbeddedFuture<>(fut,
-                    new C2<Object, Exception, IgniteInternalFuture<IgniteTxEx>>() {
-                        @Override public IgniteInternalFuture<IgniteTxEx> apply(Object o, Exception e) {
+                IgniteInternalFuture<IgniteInternalTx> f = new GridEmbeddedFuture<>(fut,
+                    new C2<Object, Exception, IgniteInternalFuture<IgniteInternalTx>>() {
+                        @Override public IgniteInternalFuture<IgniteInternalTx> apply(Object o, Exception e) {
                             return tx.commitAsync();
                         }
                     }, ctx.kernalContext());
@@ -4070,7 +4070,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
                 return f;
             }
 
-            IgniteInternalFuture<IgniteTxEx> f = tx.commitAsync();
+            IgniteInternalFuture<IgniteInternalTx> f = tx.commitAsync();
 
             saveFuture(holder, f);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
index 8eeacc5..0311fdd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
@@ -291,7 +291,7 @@ public interface GridCacheEntryEx<K, V> {
      * @throws GridCacheEntryRemovedException If entry was removed.
      * @throws GridCacheFilterFailedException If filter failed.
      */
-    @Nullable public V innerGet(@Nullable IgniteTxEx<K, V> tx,
+    @Nullable public V innerGet(@Nullable IgniteInternalTx<K, V> tx,
         boolean readSwap,
         boolean readThrough,
         boolean failFast,
@@ -341,7 +341,7 @@ public interface GridCacheEntryEx<K, V> {
      * @throws GridCacheEntryRemovedException If entry has been removed.
      */
     public GridCacheUpdateTxResult<V> innerSet(
-        @Nullable IgniteTxEx<K, V> tx,
+        @Nullable IgniteInternalTx<K, V> tx,
         UUID evtNodeId,
         UUID affNodeId,
         @Nullable V val,
@@ -380,7 +380,7 @@ public interface GridCacheEntryEx<K, V> {
      * @throws GridCacheEntryRemovedException If entry has been removed.
      */
     public GridCacheUpdateTxResult<V> innerRemove(
-        @Nullable IgniteTxEx<K, V> tx,
+        @Nullable IgniteInternalTx<K, V> tx,
         UUID evtNodeId,
         UUID affNodeId,
         boolean writeThrough,
@@ -514,7 +514,7 @@ public interface GridCacheEntryEx<K, V> {
      * @throws GridCacheEntryRemovedException If this entry is obsolete.
      * @throws GridDistributedLockCancelledException If lock has been cancelled.
      */
-    public boolean tmLock(IgniteTxEx<K, V> tx, long timeout) throws GridCacheEntryRemovedException,
+    public boolean tmLock(IgniteInternalTx<K, V> tx, long timeout) throws GridCacheEntryRemovedException,
         GridDistributedLockCancelledException;
 
     /**
@@ -523,7 +523,7 @@ public interface GridCacheEntryEx<K, V> {
      * @param tx Cache transaction.
      * @throws GridCacheEntryRemovedException If this entry has been removed from cache.
      */
-    public abstract void txUnlock(IgniteTxEx<K, V> tx) throws GridCacheEntryRemovedException;
+    public abstract void txUnlock(IgniteInternalTx<K, V> tx) throws GridCacheEntryRemovedException;
 
     /**
      * @param ver Removes lock.
@@ -623,7 +623,7 @@ public interface GridCacheEntryEx<K, V> {
      */
     @SuppressWarnings({"RedundantTypeArguments"})
     @Nullable public GridTuple<V> peek0(boolean failFast, GridCachePeekMode mode,
-        @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter, @Nullable IgniteTxEx<K, V> tx)
+        @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter, @Nullable IgniteInternalTx<K, V> tx)
         throws GridCacheEntryRemovedException, GridCacheFilterFailedException, IgniteCheckedException;
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryImpl.java
index f018a8a..05998a5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryImpl.java
@@ -277,7 +277,7 @@ public class GridCacheEntryImpl<K, V> implements CacheEntry<K, V>, Externalizabl
      */
     @SuppressWarnings({"unchecked"})
     @Nullable private V peek0(@Nullable GridCachePeekMode mode,
-        @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter, @Nullable IgniteTxEx<K, V> tx)
+        @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter, @Nullable IgniteInternalTx<K, V> tx)
         throws IgniteCheckedException {
         assert tx == null || tx.local();
 
@@ -349,7 +349,7 @@ public class GridCacheEntryImpl<K, V> implements CacheEntry<K, V>, Externalizabl
      * @throws IgniteCheckedException If failed.
      */
     @Nullable private V peek0(@Nullable Collection<GridCachePeekMode> modes,
-        @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter, IgniteTxEx<K, V> tx) throws IgniteCheckedException {
+        @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter, IgniteInternalTx<K, V> tx) throws IgniteCheckedException {
         if (F.isEmpty(modes))
             return peek0(SMART, filter, tx);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
index 25c07f0..a98b93b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
@@ -75,7 +75,7 @@ public class GridCacheEventManager<K, V> extends GridCacheManagerAdapter<K, V> {
      * @param cloClsName Closure class name.
      * @param taskName Task name.
      */
-    public void addEvent(int part, K key, IgniteTxEx tx, @Nullable GridCacheMvccCandidate<K> owner,
+    public void addEvent(int part, K key, IgniteInternalTx tx, @Nullable GridCacheMvccCandidate<K> owner,
         int type, @Nullable V newVal, boolean hasNewVal, @Nullable V oldVal, boolean hasOldVal, UUID subjId,
         String cloClsName, String taskName) {
         addEvent(part, key, locNodeId, tx, owner, type, newVal, hasNewVal, oldVal, hasOldVal, subjId, cloClsName,
@@ -97,7 +97,7 @@ public class GridCacheEventManager<K, V> extends GridCacheManagerAdapter<K, V> {
      * @param cloClsName Closure class name.
      * @param taskName Task name.
      */
-    public void addEvent(int part, K key, UUID nodeId, IgniteTxEx tx, GridCacheMvccCandidate<K> owner,
+    public void addEvent(int part, K key, UUID nodeId, IgniteInternalTx tx, GridCacheMvccCandidate<K> owner,
         int type, V newVal, boolean hasNewVal, V oldVal, boolean hasOldVal, UUID subjId, String cloClsName,
         String taskName) {
         addEvent(part, key, nodeId, tx == null ? null : tx.xid(), owner == null ? null : owner.version(), type,
@@ -121,7 +121,7 @@ public class GridCacheEventManager<K, V> extends GridCacheManagerAdapter<K, V> {
     public void addEvent(int part, K key, UUID evtNodeId, @Nullable GridCacheMvccCandidate<K> owner,
         int type, @Nullable V newVal, boolean hasNewVal, V oldVal, boolean hasOldVal, UUID subjId, String cloClsName,
         String taskName) {
-        IgniteTxEx tx = owner == null ? null : cctx.tm().tx(owner.version());
+        IgniteInternalTx tx = owner == null ? null : cctx.tm().tx(owner.version());
 
         addEvent(part, key, evtNodeId, tx == null ? null : tx.xid(), owner == null ? null : owner.version(), type,
             newVal, hasNewVal, oldVal, hasOldVal, subjId, cloClsName, taskName);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEntry.java
index 20e6e74..bb11b7d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEntry.java
@@ -152,7 +152,7 @@ public class GridCacheEvictionEntry<K, V> implements CacheEntry<K, V>, Externali
      */
     @SuppressWarnings({"unchecked"})
     @Nullable private V peek0(@Nullable GridCachePeekMode mode,
-        @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter, @Nullable IgniteTxEx<K, V> tx)
+        @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter, @Nullable IgniteInternalTx<K, V> tx)
         throws IgniteCheckedException {
         assert tx == null || tx.local();
 
@@ -184,7 +184,7 @@ public class GridCacheEvictionEntry<K, V> implements CacheEntry<K, V>, Externali
      * @throws IgniteCheckedException If failed.
      */
     @Nullable private V peek0(@Nullable Collection<GridCachePeekMode> modes,
-        @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter, IgniteTxEx<K, V> tx) throws IgniteCheckedException {
+        @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter, IgniteInternalTx<K, V> tx) throws IgniteCheckedException {
         if (F.isEmpty(modes))
             return peek0(SMART, filter, tx);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 3c4e9d6..5711ec1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -618,13 +618,13 @@ public abstract class GridCacheMapEntry<K, V> implements GridCacheEntryEx<K, V>
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings({"RedundantTypeArguments"})
-    @Nullable protected V readThrough(@Nullable IgniteTxEx<K, V> tx, K key, boolean reload,
+    @Nullable protected V readThrough(@Nullable IgniteInternalTx<K, V> tx, K key, boolean reload,
         IgnitePredicate<CacheEntry<K, V>>[] filter, UUID subjId, String taskName) throws IgniteCheckedException {
         return cctx.store().loadFromStore(tx, key);
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public final V innerGet(@Nullable IgniteTxEx<K, V> tx,
+    @Nullable @Override public final V innerGet(@Nullable IgniteInternalTx<K, V> tx,
         boolean readSwap,
         boolean readThrough,
         boolean failFast,
@@ -657,7 +657,7 @@ public abstract class GridCacheMapEntry<K, V> implements GridCacheEntryEx<K, V>
 
     /** {@inheritDoc} */
     @SuppressWarnings({"unchecked", "RedundantTypeArguments", "TooBroadScope"})
-    private V innerGet0(IgniteTxEx<K, V> tx,
+    private V innerGet0(IgniteInternalTx<K, V> tx,
         boolean readSwap,
         boolean readThrough,
         boolean evt,
@@ -853,7 +853,7 @@ public abstract class GridCacheMapEntry<K, V> implements GridCacheEntryEx<K, V>
         boolean loadedFromStore = false;
 
         if (ret == null && readThrough) {
-            IgniteTxEx tx0 = null;
+            IgniteInternalTx tx0 = null;
 
             if (tx != null && tx.local()) {
                 if (cctx.isReplicated() || cctx.isColocated() || tx.near())
@@ -1036,7 +1036,7 @@ public abstract class GridCacheMapEntry<K, V> implements GridCacheEntryEx<K, V>
 
     /** {@inheritDoc} */
     @Override public final GridCacheUpdateTxResult<V> innerSet(
-        @Nullable IgniteTxEx<K, V> tx,
+        @Nullable IgniteInternalTx<K, V> tx,
         UUID evtNodeId,
         UUID affNodeId,
         V val,
@@ -1190,7 +1190,7 @@ public abstract class GridCacheMapEntry<K, V> implements GridCacheEntryEx<K, V>
 
     /** {@inheritDoc} */
     @Override public final GridCacheUpdateTxResult<V> innerRemove(
-        @Nullable IgniteTxEx<K, V> tx,
+        @Nullable IgniteInternalTx<K, V> tx,
         UUID evtNodeId,
         UUID affNodeId,
         boolean writeThrough,
@@ -2740,7 +2740,7 @@ public abstract class GridCacheMapEntry<K, V> implements GridCacheEntryEx<K, V>
      */
     @SuppressWarnings({"RedundantTypeArguments"})
     @Nullable @Override public GridTuple<V> peek0(boolean failFast, GridCachePeekMode mode,
-        IgnitePredicate<CacheEntry<K, V>>[] filter, @Nullable IgniteTxEx<K, V> tx)
+        IgnitePredicate<CacheEntry<K, V>>[] filter, @Nullable IgniteInternalTx<K, V> tx)
         throws GridCacheEntryRemovedException, GridCacheFilterFailedException, IgniteCheckedException {
         assert tx == null || tx.local();
 
@@ -2831,7 +2831,7 @@ public abstract class GridCacheMapEntry<K, V> implements GridCacheEntryEx<K, V>
      * @throws GridCacheEntryRemovedException If entry is obsolete.
      * @throws IgniteCheckedException If entry was externally locked.
      */
-    private void groupLockSanityCheck(IgniteTxEx<K, V> tx) throws GridCacheEntryRemovedException, IgniteCheckedException {
+    private void groupLockSanityCheck(IgniteInternalTx<K, V> tx) throws GridCacheEntryRemovedException, IgniteCheckedException {
         assert tx.groupLock();
 
         IgniteTxEntry<K, V> txEntry = tx.entry(txKey());
@@ -2853,7 +2853,7 @@ public abstract class GridCacheMapEntry<K, V> implements GridCacheEntryEx<K, V>
      * @throws IgniteCheckedException If unexpected cache failure occurred.
      */
     @Nullable private GridTuple<V> peekTxThenGlobal(boolean failFast, IgnitePredicate<CacheEntry<K, V>>[] filter,
-        IgniteTxEx<K, V> tx) throws GridCacheFilterFailedException, GridCacheEntryRemovedException, IgniteCheckedException {
+        IgniteInternalTx<K, V> tx) throws GridCacheFilterFailedException, GridCacheEntryRemovedException, IgniteCheckedException {
         GridTuple<V> peek = peekTx(failFast, filter, tx);
 
         // If transaction has value (possibly null, which means value is to be deleted).
@@ -2874,7 +2874,7 @@ public abstract class GridCacheMapEntry<K, V> implements GridCacheEntryEx<K, V>
      */
     @Nullable private GridTuple<V> peekTx(boolean failFast,
         IgnitePredicate<CacheEntry<K, V>>[] filter,
-        @Nullable IgniteTxEx<K, V> tx) throws GridCacheFilterFailedException {
+        @Nullable IgniteInternalTx<K, V> tx) throws GridCacheFilterFailedException {
         return tx == null ? null : tx.peek(cctx, failFast, key, filter);
     }
 
@@ -3874,7 +3874,7 @@ public abstract class GridCacheMapEntry<K, V> implements GridCacheEntryEx<K, V>
             return false;
         }
 
-        IgniteTxEx<K, V> tx = cctx.tm().localTxx();
+        IgniteInternalTx<K, V> tx = cctx.tm().localTxx();
 
         return tx == null || !tx.removed(txKey());
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMultiTxFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMultiTxFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMultiTxFuture.java
index af5cc87..94cf5a2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMultiTxFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMultiTxFuture.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -41,10 +40,10 @@ public final class GridCacheMultiTxFuture<K, V> extends GridFutureAdapter<Boolea
     private static final AtomicReference<IgniteLogger> logRef = new AtomicReference<>();
 
     /** Transactions to wait for. */
-    private final Set<IgniteTxEx<K, V>> txs = new GridLeanSet<>();
+    private final Set<IgniteInternalTx<K, V>> txs = new GridLeanSet<>();
 
     /** */
-    private Set<IgniteTxEx<K, V>> remainingTxs;
+    private Set<IgniteInternalTx<K, V>> remainingTxs;
 
     /** Logger. */
     private IgniteLogger log;
@@ -71,21 +70,21 @@ public final class GridCacheMultiTxFuture<K, V> extends GridFutureAdapter<Boolea
     /**
      * @return Transactions to wait for.
      */
-    public Set<IgniteTxEx<K, V>> txs() {
+    public Set<IgniteInternalTx<K, V>> txs() {
         return txs;
     }
 
     /**
      * @return Remaining transactions.
      */
-    public Set<IgniteTxEx<K, V>> remainingTxs() {
+    public Set<IgniteInternalTx<K, V>> remainingTxs() {
         return remainingTxs;
     }
 
     /**
      * @param tx Transaction to add.
      */
-    public void addTx(IgniteTxEx<K, V> tx) {
+    public void addTx(IgniteInternalTx<K, V> tx) {
         txs.add(tx);
     }
 
@@ -101,10 +100,10 @@ public final class GridCacheMultiTxFuture<K, V> extends GridFutureAdapter<Boolea
         else {
             remainingTxs = new GridConcurrentHashSet<>(txs);
 
-            for (final IgniteTxEx<K, V> tx : txs) {
+            for (final IgniteInternalTx<K, V> tx : txs) {
                 if (!tx.done()) {
-                    tx.finishFuture().listenAsync(new CI1<IgniteInternalFuture<IgniteTxEx>>() {
-                        @Override public void apply(IgniteInternalFuture<IgniteTxEx> t) {
+                    tx.finishFuture().listenAsync(new CI1<IgniteInternalFuture<IgniteInternalTx>>() {
+                        @Override public void apply(IgniteInternalFuture<IgniteInternalTx> t) {
                             remainingTxs.remove(tx);
 
                             checkRemaining();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvcc.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvcc.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvcc.java
index 5489246..e208ad8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvcc.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvcc.java
@@ -806,7 +806,7 @@ public final class GridCacheMvcc<K> {
                 // Only Near and DHT remote candidates should be released.
                 assert !rmt.nearLocal();
 
-                IgniteTxEx tx = cctx.tm().tx(rmt.version());
+                IgniteInternalTx tx = cctx.tm().tx(rmt.version());
 
                 if (tx != null) {
                     tx.systemInvalidate(true);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
index 18c8c7a..c2556d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
@@ -1166,11 +1166,11 @@ public class GridCacheMvccManager<K, V> extends GridCacheSharedManagerAdapter<K,
         /** {@inheritDoc} */
         @Override public String toString() {
             if (!pendingLocks.isEmpty()) {
-                Map<GridCacheVersion, IgniteTxEx> txs = new HashMap<>(1, 1.0f);
+                Map<GridCacheVersion, IgniteInternalTx> txs = new HashMap<>(1, 1.0f);
 
                 for (Collection<GridCacheMvccCandidate<K>> cands : pendingLocks.values())
                     for (GridCacheMvccCandidate<K> c : cands)
-                        txs.put(c.version(), cctx.tm().<IgniteTxEx>tx(c.version()));
+                        txs.put(c.version(), cctx.tm().<IgniteInternalTx>tx(c.version()));
 
                 return S.toString(FinishLockFuture.class, this, "txs=" + txs + ", super=" + super.toString());
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java
index ebd6f81..3346f0b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java
@@ -1240,7 +1240,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteTxEx txStartEx(IgniteTxConcurrency concurrency, IgniteTxIsolation isolation) {
+    @Override public IgniteInternalTx txStartEx(IgniteTxConcurrency concurrency, IgniteTxIsolation isolation) {
         return cache.txStartEx(concurrency, isolation);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
index 0dfea35..7a02c50 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
@@ -1129,7 +1129,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteTxEx txStartEx(IgniteTxConcurrency concurrency, IgniteTxIsolation isolation) {
+    @Override public IgniteInternalTx txStartEx(IgniteTxConcurrency concurrency, IgniteTxIsolation isolation) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/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 c036de4..64d4ee1 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
@@ -23,7 +23,6 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.marshaller.*;
-import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.managers.deployment.*;
 import org.apache.ignite.internal.managers.discovery.*;
@@ -399,7 +398,7 @@ public class GridCacheSharedContext<K, V> {
      * @param cacheCtx Cache context.
      * @return {@code True} if cross-cache transaction can include this new cache.
      */
-    public boolean txCompatible(IgniteTxEx<K, V> tx, Iterable<Integer> activeCacheIds, GridCacheContext<K, V> cacheCtx) {
+    public boolean txCompatible(IgniteInternalTx<K, V> tx, Iterable<Integer> activeCacheIds, GridCacheContext<K, V> cacheCtx) {
         if (cacheCtx.system() ^ tx.system())
             return false;
 
@@ -419,7 +418,7 @@ public class GridCacheSharedContext<K, V> {
      * @throws CacheFlagException If given flags are conflicting with given transaction.
      */
     public void checkTxFlags(@Nullable Collection<CacheFlag> flags) throws CacheFlagException {
-        IgniteTxEx tx = tm().userTxx();
+        IgniteInternalTx tx = tm().userTxx();
 
         if (tx == null || F.isEmpty(flags))
             return;
@@ -446,7 +445,7 @@ public class GridCacheSharedContext<K, V> {
      * @param tx Transaction to close.
      * @throws IgniteCheckedException If failed.
      */
-    public void endTx(IgniteTxEx<K, V> tx) throws IgniteCheckedException {
+    public void endTx(IgniteInternalTx<K, V> tx) throws IgniteCheckedException {
         Collection<Integer> cacheIds = tx.activeCacheIds();
 
         if (!cacheIds.isEmpty()) {
@@ -461,7 +460,7 @@ public class GridCacheSharedContext<K, V> {
      * @param tx Transaction to commit.
      * @return Commit future.
      */
-    public IgniteInternalFuture<IgniteTxEx> commitTxAsync(IgniteTxEx<K, V> tx) {
+    public IgniteInternalFuture<IgniteInternalTx> commitTxAsync(IgniteInternalTx<K, V> tx) {
         Collection<Integer> cacheIds = tx.activeCacheIds();
 
         if (cacheIds.isEmpty())
@@ -483,7 +482,7 @@ public class GridCacheSharedContext<K, V> {
      * @param tx Transaction to rollback.
      * @throws IgniteCheckedException If failed.
      */
-    public IgniteInternalFuture rollbackTxAsync(IgniteTxEx<K, V> tx) throws IgniteCheckedException {
+    public IgniteInternalFuture rollbackTxAsync(IgniteInternalTx<K, V> tx) throws IgniteCheckedException {
         Collection<Integer> cacheIds = tx.activeCacheIds();
 
         if (!cacheIds.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManager.java
index 98c9ad3..3f15fe5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManager.java
@@ -249,7 +249,7 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
      * @throws IgniteCheckedException If data loading failed.
      */
     @SuppressWarnings("unchecked")
-    @Nullable public V loadFromStore(@Nullable IgniteTxEx tx, K key) throws IgniteCheckedException {
+    @Nullable public V loadFromStore(@Nullable IgniteInternalTx tx, K key) throws IgniteCheckedException {
         return (V)loadFromStore(tx, key, true);
     }
 
@@ -263,7 +263,7 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
      * @throws IgniteCheckedException If data loading failed.
      */
     @SuppressWarnings("unchecked")
-    @Nullable private Object loadFromStore(@Nullable IgniteTxEx tx,
+    @Nullable private Object loadFromStore(@Nullable IgniteInternalTx tx,
         K key,
         boolean convert)
         throws IgniteCheckedException {
@@ -338,7 +338,7 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
      * @param vis Closure to apply for loaded elements.
      * @throws IgniteCheckedException If data loading failed.
      */
-    public void localStoreLoadAll(@Nullable IgniteTxEx tx,
+    public void localStoreLoadAll(@Nullable IgniteInternalTx tx,
         Collection<? extends K> keys,
         final GridInClosure3<K, V, GridCacheVersion> vis)
         throws IgniteCheckedException {
@@ -358,7 +358,7 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
      * @throws IgniteCheckedException If data loading failed.
      */
     @SuppressWarnings({"unchecked"})
-    public boolean loadAllFromStore(@Nullable IgniteTxEx tx,
+    public boolean loadAllFromStore(@Nullable IgniteInternalTx tx,
         Collection<? extends K> keys,
         final IgniteBiInClosure<K, V> vis) throws IgniteCheckedException {
         if (store != null) {
@@ -382,7 +382,7 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
-    private void loadAllFromStore(@Nullable IgniteTxEx tx,
+    private void loadAllFromStore(@Nullable IgniteInternalTx tx,
         Collection<? extends K> keys,
         final @Nullable IgniteBiInClosure<K, V> vis,
         final @Nullable GridInClosure3<K, V, GridCacheVersion> verVis)
@@ -538,7 +538,7 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
      * @throws IgniteCheckedException If storage failed.
      */
     @SuppressWarnings("unchecked")
-    public boolean putToStore(@Nullable IgniteTxEx tx, K key, V val, GridCacheVersion ver)
+    public boolean putToStore(@Nullable IgniteInternalTx tx, K key, V val, GridCacheVersion ver)
         throws IgniteCheckedException {
         if (store != null) {
             // Never persist internal keys.
@@ -588,7 +588,7 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
      * @return {@code True} if there is a persistent storage.
      * @throws IgniteCheckedException If storage failed.
      */
-    public boolean putAllToStore(@Nullable IgniteTxEx tx, Map<K, IgniteBiTuple<V, GridCacheVersion>> map)
+    public boolean putAllToStore(@Nullable IgniteInternalTx tx, Map<K, IgniteBiTuple<V, GridCacheVersion>> map)
         throws IgniteCheckedException {
         if (F.isEmpty(map))
             return true;
@@ -649,7 +649,7 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
      * @throws IgniteCheckedException If storage failed.
      */
     @SuppressWarnings("unchecked")
-    public boolean removeFromStore(@Nullable IgniteTxEx tx, K key) throws IgniteCheckedException {
+    public boolean removeFromStore(@Nullable IgniteInternalTx tx, K key) throws IgniteCheckedException {
         if (store != null) {
             // Never remove internal key from store as it is never persisted.
             if (key instanceof GridCacheInternal)
@@ -695,7 +695,7 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
      * @throws IgniteCheckedException If storage failed.
      */
     @SuppressWarnings("unchecked")
-    public boolean removeAllFromStore(@Nullable IgniteTxEx tx, Collection<?> keys) throws IgniteCheckedException {
+    public boolean removeAllFromStore(@Nullable IgniteInternalTx tx, Collection<?> keys) throws IgniteCheckedException {
         if (F.isEmpty(keys))
             return true;
 
@@ -762,7 +762,7 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
      * @param commit Commit.
      * @throws IgniteCheckedException If failed.
      */
-    public void txEnd(IgniteTxEx tx, boolean commit) throws IgniteCheckedException {
+    public void txEnd(IgniteInternalTx tx, boolean commit) throws IgniteCheckedException {
         assert store != null;
 
         initSession(tx);
@@ -796,7 +796,7 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
     /**
      * @param tx Current transaction.
      */
-    private void initSession(@Nullable IgniteTxEx<?, ?> tx) {
+    private void initSession(@Nullable IgniteInternalTx<?, ?> tx) {
         SessionData ses;
 
         if (tx != null) {
@@ -823,7 +823,7 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
     private static class SessionData {
         /** */
         @GridToStringExclude
-        private final IgniteTxEx tx;
+        private final IgniteInternalTx tx;
 
         /** */
         private String cacheName;
@@ -836,7 +836,7 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
          * @param tx Current transaction.
          * @param cacheName Cache name.
          */
-        private SessionData(@Nullable IgniteTxEx tx, @Nullable String cacheName) {
+        private SessionData(@Nullable IgniteInternalTx tx, @Nullable String cacheName) {
             this.tx = tx;
             this.cacheName = cacheName;
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/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 7c5e732..12200eb 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
@@ -167,8 +167,8 @@ public class GridCacheUtils {
     };
 
     /** Converts transaction to XID version. */
-    private static final IgniteClosure tx2xidVer = new C1<IgniteTxEx, GridCacheVersion>() {
-        @Override public GridCacheVersion apply(IgniteTxEx tx) {
+    private static final IgniteClosure tx2xidVer = new C1<IgniteInternalTx, GridCacheVersion>() {
+        @Override public GridCacheVersion apply(IgniteInternalTx tx) {
             return tx.xidVersion();
         }
 
@@ -790,8 +790,8 @@ public class GridCacheUtils {
      * @return Closure which converts transaction entry xid to XID version.
      */
     @SuppressWarnings( {"unchecked"})
-    public static <K, V> IgniteClosure<IgniteTxEx<K, V>, GridCacheVersion> tx2xidVersion() {
-        return (IgniteClosure<IgniteTxEx<K, V>, GridCacheVersion>)tx2xidVer;
+    public static <K, V> IgniteClosure<IgniteInternalTx<K, V>, GridCacheVersion> tx2xidVersion() {
+        return (IgniteClosure<IgniteInternalTx<K, V>, GridCacheVersion>)tx2xidVer;
     }
 
     /**
@@ -1192,7 +1192,7 @@ public class GridCacheUtils {
      * @param isolation Isolation.
      * @return New transaction.
      */
-    public static IgniteTxEx txStartInternal(GridCacheContext ctx, CacheProjection prj,
+    public static IgniteInternalTx txStartInternal(GridCacheContext ctx, CacheProjection prj,
         IgniteTxConcurrency concurrency, IgniteTxIsolation isolation) {
         assert ctx != null;
         assert prj != null;
@@ -1206,7 +1206,7 @@ public class GridCacheUtils {
      * @param tx Transaction.
      * @return String view of all safe-to-print transaction properties.
      */
-    public static String txString(@Nullable IgniteTxEx tx) {
+    public static String txString(@Nullable IgniteInternalTx tx) {
         if (tx == null)
             return "null";
 
@@ -1609,7 +1609,7 @@ public class GridCacheUtils {
     public static <K, V> void inTx(CacheProjection<K, V> cache, IgniteTxConcurrency concurrency,
         IgniteTxIsolation isolation, IgniteInClosureX<CacheProjection<K ,V>> clo) throws IgniteCheckedException {
 
-        try (IgniteTxEx tx = cache.txStartEx(concurrency, isolation);) {
+        try (IgniteInternalTx tx = cache.txStartEx(concurrency, isolation);) {
             clo.applyx(cache);
 
             tx.commit();
@@ -1622,7 +1622,7 @@ public class GridCacheUtils {
      * @param tx Transaction.
      * @return Subject ID.
      */
-    public static <K, V> UUID subjectId(IgniteTxEx<K, V> tx, GridCacheSharedContext<K, V> ctx) {
+    public static <K, V> UUID subjectId(IgniteInternalTx<K, V> tx, GridCacheSharedContext<K, V> ctx) {
         if (tx == null)
             return ctx.localNodeId();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicLongImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicLongImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicLongImpl.java
index 3157674..e791797 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicLongImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicLongImpl.java
@@ -23,7 +23,6 @@ import org.apache.ignite.cache.datastructures.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.lang.*;
-import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
@@ -82,7 +81,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ext
     /** Callable for {@link #incrementAndGet()}. */
     private final Callable<Long> incAndGetCall = new Callable<Long>() {
         @Override public Long call() throws Exception {
-            IgniteTxEx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
+            IgniteInternalTx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
 
             try {
                 GridCacheAtomicLongValue val = atomicView.get(key);
@@ -113,7 +112,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ext
     /** Callable for {@link #getAndIncrement()}. */
     private final Callable<Long> getAndIncCall = new Callable<Long>() {
         @Override public Long call() throws Exception {
-            IgniteTxEx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
+            IgniteInternalTx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
 
             try {
                 GridCacheAtomicLongValue val = atomicView.get(key);
@@ -144,7 +143,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ext
     /** Callable for {@link #decrementAndGet()}. */
     private final Callable<Long> decAndGetCall = new Callable<Long>() {
         @Override public Long call() throws Exception {
-            IgniteTxEx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
+            IgniteInternalTx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
 
             try {
                 GridCacheAtomicLongValue val = atomicView.get(key);
@@ -175,7 +174,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ext
     /** Callable for {@link #getAndDecrement()}. */
     private final Callable<Long> getAndDecCall = new Callable<Long>() {
         @Override public Long call() throws Exception {
-            IgniteTxEx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
+            IgniteInternalTx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
 
             try {
                 GridCacheAtomicLongValue val = atomicView.get(key);
@@ -340,7 +339,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ext
     private Callable<Long> internalAddAndGet(final long l) {
         return new Callable<Long>() {
             @Override public Long call() throws Exception {
-                IgniteTxEx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
+                IgniteInternalTx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
 
                 try {
                     GridCacheAtomicLongValue val = atomicView.get(key);
@@ -378,7 +377,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ext
     private Callable<Long> internalGetAndAdd(final long l) {
         return new Callable<Long>() {
             @Override public Long call() throws Exception {
-                IgniteTxEx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
+                IgniteInternalTx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
 
                 try {
                     GridCacheAtomicLongValue val = atomicView.get(key);
@@ -416,7 +415,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ext
     private Callable<Long> internalGetAndSet(final long l) {
         return new Callable<Long>() {
             @Override public Long call() throws Exception {
-                IgniteTxEx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
+                IgniteInternalTx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
 
                 try {
                     GridCacheAtomicLongValue val = atomicView.get(key);
@@ -456,7 +455,7 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ext
     private Callable<Boolean> internalCompareAndSet(final long expVal, final long newVal) {
         return new Callable<Boolean>() {
             @Override public Boolean call() throws Exception {
-                IgniteTxEx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
+                IgniteInternalTx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
 
                 try {
                     GridCacheAtomicLongValue val = atomicView.get(key);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicReferenceImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicReferenceImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicReferenceImpl.java
index 72504a9..40ac895 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicReferenceImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicReferenceImpl.java
@@ -23,7 +23,6 @@ import org.apache.ignite.cache.datastructures.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.lang.*;
-import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
@@ -194,7 +193,7 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
     private Callable<Boolean> internalSet(final T val) {
         return new Callable<Boolean>() {
             @Override public Boolean call() throws Exception {
-                IgniteTxEx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
+                IgniteInternalTx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
 
                 try {
                     GridCacheAtomicReferenceValue<T> ref = atomicView.get(key);
@@ -233,7 +232,7 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
         final IgniteClosure<T, T> newValClos) {
         return new Callable<Boolean>() {
             @Override public Boolean call() throws Exception {
-                IgniteTxEx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
+                IgniteInternalTx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
 
                 try {
                     GridCacheAtomicReferenceValue<T> ref = atomicView.get(key);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicSequenceImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicSequenceImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicSequenceImpl.java
index c8c46aa..6c6c890 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicSequenceImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicSequenceImpl.java
@@ -24,7 +24,6 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.lang.*;
-import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.util.future.*;
@@ -401,7 +400,7 @@ public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenc
     private Callable<Long> internalUpdate(final long l, final boolean updated) {
         return new Callable<Long>() {
             @Override public Long call() throws Exception {
-                IgniteTxEx tx = CU.txStartInternal(ctx, seqView, PESSIMISTIC, REPEATABLE_READ);
+                IgniteInternalTx tx = CU.txStartInternal(ctx, seqView, PESSIMISTIC, REPEATABLE_READ);
 
                 try {
                     GridCacheAtomicSequenceValue seq = seqView.get(key);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicStampedImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicStampedImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicStampedImpl.java
index bf5861a..01764dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicStampedImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicStampedImpl.java
@@ -24,7 +24,6 @@ import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.lang.*;
-import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -220,7 +219,7 @@ public final class GridCacheAtomicStampedImpl<T, S> implements GridCacheAtomicSt
     private Callable<Boolean> internalSet(final T val, final S stamp) {
         return new Callable<Boolean>() {
             @Override public Boolean call() throws Exception {
-                IgniteTxEx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
+                IgniteInternalTx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
 
                 try {
                     GridCacheAtomicStampedValue<T, S> stmp = atomicView.get(key);
@@ -262,7 +261,7 @@ public final class GridCacheAtomicStampedImpl<T, S> implements GridCacheAtomicSt
         final IgniteClosure<S, S> newStampClos) {
         return new Callable<Boolean>() {
             @Override public Boolean call() throws Exception {
-                IgniteTxEx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
+                IgniteInternalTx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ);
 
                 try {
                     GridCacheAtomicStampedValue<T, S> stmp = atomicView.get(key);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheCountDownLatchImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheCountDownLatchImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheCountDownLatchImpl.java
index 018d8d5..6e9dfa8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheCountDownLatchImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheCountDownLatchImpl.java
@@ -219,7 +219,7 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
                 internalLatch = CU.outTx(
                     new Callable<CountDownLatch>() {
                         @Override public CountDownLatch call() throws Exception {
-                            IgniteTxEx tx = CU.txStartInternal(ctx, latchView, PESSIMISTIC, REPEATABLE_READ);
+                            IgniteInternalTx tx = CU.txStartInternal(ctx, latchView, PESSIMISTIC, REPEATABLE_READ);
 
                             try {
                                 GridCacheCountDownLatchValue val = latchView.get(key);
@@ -318,7 +318,7 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
 
         /** {@inheritDoc} */
         @Override public Integer call() throws Exception {
-            IgniteTxEx tx = CU.txStartInternal(ctx, latchView, PESSIMISTIC, REPEATABLE_READ);
+            IgniteInternalTx tx = CU.txStartInternal(ctx, latchView, PESSIMISTIC, REPEATABLE_READ);
 
             try {
                 GridCacheCountDownLatchValue latchVal = latchView.get(key);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresManager.java
index 593459f..0537407 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresManager.java
@@ -199,7 +199,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
 
             return CU.outTx(new Callable<CacheAtomicSequence>() {
                 @Override public CacheAtomicSequence call() throws Exception {
-                    try (IgniteTxEx tx = CU.txStartInternal(cctx, dsView, PESSIMISTIC, REPEATABLE_READ)) {
+                    try (IgniteInternalTx tx = CU.txStartInternal(cctx, dsView, PESSIMISTIC, REPEATABLE_READ)) {
                         GridCacheAtomicSequenceValue seqVal = cast(dsView.get(key),
                             GridCacheAtomicSequenceValue.class);
 
@@ -317,7 +317,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
             return CU.outTx(new Callable<CacheAtomicLong>() {
                 @Override
                 public CacheAtomicLong call() throws Exception {
-                    try (IgniteTxEx tx = CU.txStartInternal(cctx, dsView, PESSIMISTIC, REPEATABLE_READ)) {
+                    try (IgniteInternalTx tx = CU.txStartInternal(cctx, dsView, PESSIMISTIC, REPEATABLE_READ)) {
                         GridCacheAtomicLongValue val = cast(dsView.get(key),
                             GridCacheAtomicLongValue.class);
 
@@ -413,7 +413,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
             return CU.outTx(new Callable<CacheAtomicReference<T>>() {
                 @Override
                 public CacheAtomicReference<T> call() throws Exception {
-                    try (IgniteTxEx tx = CU.txStartInternal(cctx, dsView, PESSIMISTIC, REPEATABLE_READ)) {
+                    try (IgniteInternalTx tx = CU.txStartInternal(cctx, dsView, PESSIMISTIC, REPEATABLE_READ)) {
                         GridCacheAtomicReferenceValue val = cast(dsView.get(key),
                             GridCacheAtomicReferenceValue.class);
 
@@ -512,7 +512,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
             return CU.outTx(new Callable<CacheAtomicStamped<T, S>>() {
                 @Override
                 public CacheAtomicStamped<T, S> call() throws Exception {
-                    try (IgniteTxEx tx = CU.txStartInternal(cctx, dsView, PESSIMISTIC, REPEATABLE_READ)) {
+                    try (IgniteInternalTx tx = CU.txStartInternal(cctx, dsView, PESSIMISTIC, REPEATABLE_READ)) {
                         GridCacheAtomicStampedValue val = cast(dsView.get(key),
                             GridCacheAtomicStampedValue.class);
 
@@ -792,7 +792,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
 
             return CU.outTx(new Callable<CacheCountDownLatch>() {
                     @Override public CacheCountDownLatch call() throws Exception {
-                        try (IgniteTxEx tx = CU.txStartInternal(cctx, dsView, PESSIMISTIC, REPEATABLE_READ)) {
+                        try (IgniteInternalTx tx = CU.txStartInternal(cctx, dsView, PESSIMISTIC, REPEATABLE_READ)) {
                             GridCacheCountDownLatchValue val = cast(dsView.get(key),
                                 GridCacheCountDownLatchValue.class);
 
@@ -856,7 +856,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
                     @Override public Boolean call() throws Exception {
                         GridCacheInternal key = new GridCacheInternalKeyImpl(name);
 
-                        try (IgniteTxEx tx = CU.txStartInternal(cctx, dsView, PESSIMISTIC, REPEATABLE_READ)) {
+                        try (IgniteInternalTx tx = CU.txStartInternal(cctx, dsView, PESSIMISTIC, REPEATABLE_READ)) {
                             // Check correctness type of removable object.
                             GridCacheCountDownLatchValue val =
                                 cast(dsView.get(key), GridCacheCountDownLatchValue.class);
@@ -903,7 +903,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
         return CU.outTx(
             new Callable<Boolean>() {
                 @Override public Boolean call() throws Exception {
-                    try (IgniteTxEx tx = CU.txStartInternal(cctx, dsView, PESSIMISTIC, REPEATABLE_READ)) {
+                    try (IgniteInternalTx tx = CU.txStartInternal(cctx, dsView, PESSIMISTIC, REPEATABLE_READ)) {
                         // Check correctness type of removable object.
                         R val = cast(dsView.get(key), cls);
 
@@ -933,7 +933,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
      *
      * @param tx Committed transaction.
      */
-    public void onTxCommitted(IgniteTxEx<K, V> tx) {
+    public void onTxCommitted(IgniteInternalTx<K, V> tx) {
         if (!cctx.isDht() && tx.internal() && (!cctx.isColocated() || cctx.isReplicated())) {
             try {
                 waitInitialization();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridTransactionalCacheQueueImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridTransactionalCacheQueueImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridTransactionalCacheQueueImpl.java
index 9340380..861bbd5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridTransactionalCacheQueueImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridTransactionalCacheQueueImpl.java
@@ -24,7 +24,6 @@ import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
-import javax.cache.*;
 import java.util.*;
 
 import static org.apache.ignite.transactions.IgniteTxConcurrency.*;
@@ -55,7 +54,7 @@ public class GridTransactionalCacheQueueImpl<T> extends GridCacheQueueAdapter<T>
 
             while (true) {
                 try {
-                    try (IgniteTxEx tx = cache.txStartEx(PESSIMISTIC, REPEATABLE_READ)) {
+                    try (IgniteInternalTx tx = cache.txStartEx(PESSIMISTIC, REPEATABLE_READ)) {
                         Long idx = (Long)cache.invoke(queueKey, new AddProcessor(id, 1)).get();
 
                         if (idx != null) {
@@ -103,7 +102,7 @@ public class GridTransactionalCacheQueueImpl<T> extends GridCacheQueueAdapter<T>
             T retVal;
 
             while (true) {
-                try (IgniteTxEx tx = cache.txStartEx(PESSIMISTIC, REPEATABLE_READ)) {
+                try (IgniteInternalTx tx = cache.txStartEx(PESSIMISTIC, REPEATABLE_READ)) {
                     Long idx = (Long)cache.invoke(queueKey, new PollProcessor(id)).get();
 
                     if (idx != null) {
@@ -152,7 +151,7 @@ public class GridTransactionalCacheQueueImpl<T> extends GridCacheQueueAdapter<T>
             int cnt = 0;
 
             while (true) {
-                try (IgniteTxEx tx = cache.txStartEx(PESSIMISTIC, REPEATABLE_READ)) {
+                try (IgniteInternalTx tx = cache.txStartEx(PESSIMISTIC, REPEATABLE_READ)) {
                     Long idx = (Long)cache.invoke(queueKey, new AddProcessor(id, items.size())).get();
 
                     if (idx != null) {
@@ -205,7 +204,7 @@ public class GridTransactionalCacheQueueImpl<T> extends GridCacheQueueAdapter<T>
             int cnt = 0;
 
             while (true) {
-                try (IgniteTxEx tx = cache.txStartEx(PESSIMISTIC, REPEATABLE_READ)) {
+                try (IgniteInternalTx tx = cache.txStartEx(PESSIMISTIC, REPEATABLE_READ)) {
                     Long idx = (Long)cache.invoke(queueKey, new RemoveProcessor(id, rmvIdx)).get();
 
                     if (idx != null) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheCommittedTxInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheCommittedTxInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheCommittedTxInfo.java
index 8609259..72fb002 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheCommittedTxInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheCommittedTxInfo.java
@@ -53,7 +53,7 @@ public class GridCacheCommittedTxInfo<K, V> implements Externalizable {
     /**
      * @param tx Committed cache transaction.
      */
-    public GridCacheCommittedTxInfo(IgniteTxEx<K, V> tx) {
+    public GridCacheCommittedTxInfo(IgniteInternalTx<K, V> tx) {
         assert !tx.local() || !tx.replicated();
 
         originatingTxId = tx.nearXidVersion();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxFuture.java
index 21fbffa..958d557 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxFuture.java
@@ -56,7 +56,7 @@ public class GridCacheOptimisticCheckPreparedTxFuture<K, V> extends GridCompound
     private final IgniteUuid futId = IgniteUuid.randomUuid();
 
     /** Transaction. */
-    private final IgniteTxEx<K, V> tx;
+    private final IgniteInternalTx<K, V> tx;
 
     /** All involved nodes. */
     private final Map<UUID, ClusterNode> nodes;
@@ -77,7 +77,7 @@ public class GridCacheOptimisticCheckPreparedTxFuture<K, V> extends GridCompound
      * @param txNodes Transaction mapping.
      */
     @SuppressWarnings("ConstantConditions")
-    public GridCacheOptimisticCheckPreparedTxFuture(GridCacheSharedContext<K, V> cctx, IgniteTxEx<K, V> tx,
+    public GridCacheOptimisticCheckPreparedTxFuture(GridCacheSharedContext<K, V> cctx, IgniteInternalTx<K, V> tx,
         UUID failedNodeId, Map<UUID, Collection<UUID>> txNodes) {
         super(cctx.kernalContext(), CU.boolReducer());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxRequest.java
index 73ad5d0..41144b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxRequest.java
@@ -59,7 +59,7 @@ public class GridCacheOptimisticCheckPreparedTxRequest<K, V> extends GridDistrib
      * @param futId Future ID.
      * @param miniId Mini future ID.
      */
-    public GridCacheOptimisticCheckPreparedTxRequest(IgniteTxEx<K, V> tx, int txNum, IgniteUuid futId, IgniteUuid miniId) {
+    public GridCacheOptimisticCheckPreparedTxRequest(IgniteInternalTx<K, V> tx, int txNum, IgniteUuid futId, IgniteUuid miniId) {
         super(tx.xidVersion(), 0);
 
         nearXidVer = tx.nearXidVersion();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePerThreadTxCommitBuffer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePerThreadTxCommitBuffer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePerThreadTxCommitBuffer.java
index e6c7e5c..120127a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePerThreadTxCommitBuffer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePerThreadTxCommitBuffer.java
@@ -58,7 +58,7 @@ public class GridCachePerThreadTxCommitBuffer<K, V> implements GridCacheTxCommit
     }
 
     /** {@inheritDoc} */
-    @Override public void addCommittedTx(IgniteTxEx<K, V> tx) {
+    @Override public void addCommittedTx(IgniteInternalTx<K, V> tx) {
         long threadId = tx.threadId();
 
         StoreKey key = new StoreKey(tx.eventNodeId(), threadId);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxFuture.java
index 2c6cc51..730c358 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxFuture.java
@@ -54,7 +54,7 @@ public class GridCachePessimisticCheckCommittedTxFuture<K, V> extends GridCompou
     private final IgniteUuid futId = IgniteUuid.randomUuid();
 
     /** Transaction. */
-    private final IgniteTxEx<K, V> tx;
+    private final IgniteInternalTx<K, V> tx;
 
     /** All involved nodes. */
     private final Map<UUID, ClusterNode> nodes;
@@ -71,7 +71,7 @@ public class GridCachePessimisticCheckCommittedTxFuture<K, V> extends GridCompou
      * @param failedNodeId ID of failed node started transaction.
      */
     @SuppressWarnings("ConstantConditions")
-    public GridCachePessimisticCheckCommittedTxFuture(GridCacheSharedContext<K, V> cctx, IgniteTxEx<K, V> tx,
+    public GridCachePessimisticCheckCommittedTxFuture(GridCacheSharedContext<K, V> cctx, IgniteInternalTx<K, V> tx,
         UUID failedNodeId) {
         super(cctx.kernalContext(), new SingleReducer<K, V>());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxRequest.java
index 2aae468..9280e48 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxRequest.java
@@ -67,7 +67,7 @@ public class GridCachePessimisticCheckCommittedTxRequest<K, V> extends GridDistr
      * @param originatingThreadId Originating thread ID.
      * @param futId Future ID.
      */
-    public GridCachePessimisticCheckCommittedTxRequest(IgniteTxEx<K, V> tx, long originatingThreadId, IgniteUuid futId,
+    public GridCachePessimisticCheckCommittedTxRequest(IgniteInternalTx<K, V> tx, long originatingThreadId, IgniteUuid futId,
         boolean nearOnlyCheck) {
         super(tx.xidVersion(), 0);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxCommitBuffer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxCommitBuffer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxCommitBuffer.java
index dd099db..54342d7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxCommitBuffer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxCommitBuffer.java
@@ -32,7 +32,7 @@ public interface GridCacheTxCommitBuffer<K, V> {
      *
      * @param tx Committed transaction.
      */
-    public void addCommittedTx(IgniteTxEx<K, V> tx);
+    public void addCommittedTx(IgniteInternalTx<K, V> tx);
 
     /**
      * Gets transaction from commit buffer.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
index 078e84e..9f32d14 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
@@ -721,7 +721,7 @@ public class GridDistributedCacheEntry<K, V> extends GridCacheMapEntry<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean tmLock(IgniteTxEx<K, V> tx, long timeout)
+    @Override public boolean tmLock(IgniteInternalTx<K, V> tx, long timeout)
         throws GridCacheEntryRemovedException, GridDistributedLockCancelledException {
         if (tx.local())
             // Null is returned if timeout is negative and there is other lock owner.
@@ -756,7 +756,7 @@ public class GridDistributedCacheEntry<K, V> extends GridCacheMapEntry<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public void txUnlock(IgniteTxEx<K, V> tx) throws GridCacheEntryRemovedException {
+    @Override public void txUnlock(IgniteInternalTx<K, V> tx) throws GridCacheEntryRemovedException {
         removeLock(tx.xidVersion());
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java
index 7d5741f..9dbeacd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java
@@ -132,7 +132,7 @@ public class GridDistributedTxPrepareRequest<K, V> extends GridDistributedBaseMe
      * @param txNodes Transaction nodes mapping.
      */
     public GridDistributedTxPrepareRequest(
-        IgniteTxEx<K, V> tx,
+        IgniteInternalTx<K, V> tx,
         @Nullable Collection<IgniteTxEntry<K, V>> reads,
         Collection<IgniteTxEntry<K, V>> writes,
         IgniteTxKey grpLockKey,


[3/4] incubator-ignite git commit: # ignite-26 renamed IgniteTxEx - IgniteInternalTx

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/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 8dae033..f9e2342 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
@@ -360,7 +360,7 @@ public class GridDistributedTxRemoteAdapter<K, V> extends IgniteTxAdapter<K, V>
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<IgniteTxEx<K, V>> prepareAsync() {
+    @Override public IgniteInternalFuture<IgniteInternalTx<K, V>> prepareAsync() {
         assert false;
         return null;
     }
@@ -718,11 +718,11 @@ public class GridDistributedTxRemoteAdapter<K, V> extends IgniteTxAdapter<K, V>
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<IgniteTxEx> commitAsync() {
+    @Override public IgniteInternalFuture<IgniteInternalTx> commitAsync() {
         try {
             commit();
 
-            return new GridFinishedFutureEx<IgniteTxEx>(this);
+            return new GridFinishedFutureEx<IgniteInternalTx>(this);
         }
         catch (IgniteCheckedException e) {
             return new GridFinishedFutureEx<>(e);
@@ -749,10 +749,10 @@ public class GridDistributedTxRemoteAdapter<K, V> extends IgniteTxAdapter<K, V>
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<IgniteTxEx> rollbackAsync() {
+    @Override public IgniteInternalFuture<IgniteInternalTx> rollbackAsync() {
         rollback();
 
-        return new GridFinishedFutureEx<IgniteTxEx>(this);
+        return new GridFinishedFutureEx<IgniteInternalTx>(this);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
index 550a693..2b244a8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
@@ -234,7 +234,7 @@ public class GridDhtCacheEntry<K, V> extends GridDistributedCacheEntry<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean tmLock(IgniteTxEx<K, V> tx, long timeout)
+    @Override public boolean tmLock(IgniteInternalTx<K, V> tx, long timeout)
         throws GridCacheEntryRemovedException, GridDistributedLockCancelledException {
         if (tx.local()) {
             GridDhtTxLocalAdapter<K, V> dhtTx = (GridDhtTxLocalAdapter<K, V>)tx;
@@ -436,7 +436,7 @@ public class GridDhtCacheEntry<K, V> extends GridDistributedCacheEntry<K, V> {
 
             if (!F.isEmpty(cands)) {
                 for (GridCacheMvccCandidate<K> c : cands) {
-                    IgniteTxEx<K, V> tx = cctx.tm().tx(c.version());
+                    IgniteInternalTx<K, V> tx = cctx.tm().tx(c.version());
 
                     if (tx != null) {
                         assert tx.local();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
index c6474ed..57c3455 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
@@ -869,8 +869,8 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
                                             assert t.implicit();
 
                                             return t.commitAsync().chain(
-                                                new C1<IgniteInternalFuture<IgniteTxEx>, GridNearLockResponse<K, V>>() {
-                                                    @Override public GridNearLockResponse<K, V> apply(IgniteInternalFuture<IgniteTxEx> f) {
+                                                new C1<IgniteInternalFuture<IgniteInternalTx>, GridNearLockResponse<K, V>>() {
+                                                    @Override public GridNearLockResponse<K, V> apply(IgniteInternalFuture<IgniteInternalTx> f) {
                                                         try {
                                                             // Check for error.
                                                             f.get();
@@ -1096,7 +1096,7 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
      */
     private void sendLockReply(
         ClusterNode nearNode,
-        @Nullable IgniteTxEx<K,V> tx,
+        @Nullable IgniteInternalTx<K,V> tx,
         GridNearLockRequest<K, V> req,
         GridNearLockResponse<K, V> res
     ) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/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 fbb2a69..ac2b55a 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
@@ -43,8 +43,8 @@ import static org.apache.ignite.internal.managers.communication.GridIoPolicy.*;
 /**
  *
  */
-public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFuture<IgniteTxEx>
-    implements GridCacheFuture<IgniteTxEx> {
+public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFuture<IgniteInternalTx>
+    implements GridCacheFuture<IgniteInternalTx> {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -93,7 +93,7 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
      * @param commit Commit flag.
      */
     public GridDhtTxFinishFuture(GridCacheSharedContext<K, V> cctx, GridDhtTxLocalAdapter<K, V> tx, boolean commit) {
-        super(cctx.kernalContext(), F.<IgniteTxEx>identityReducer(tx));
+        super(cctx.kernalContext(), F.<IgniteInternalTx>identityReducer(tx));
 
         assert cctx != null;
 
@@ -201,7 +201,7 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
      */
     public void onResult(UUID nodeId, GridDhtTxFinishResponse<K, V> res) {
         if (!isDone()) {
-            for (IgniteInternalFuture<IgniteTxEx> fut : futures()) {
+            for (IgniteInternalFuture<IgniteInternalTx> fut : futures()) {
                 if (isMini(fut)) {
                     MiniFuture f = (MiniFuture)fut;
 
@@ -216,7 +216,7 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onDone(IgniteTxEx tx, Throwable err) {
+    @Override public boolean onDone(IgniteInternalTx tx, Throwable err) {
         if (initialized() || err != null) {
             if (this.tx.onePhaseCommit() && (this.tx.state() == COMMITTING))
                 this.tx.tmCommit();
@@ -446,7 +446,7 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
      * Mini-future for get operations. Mini-futures are only waiting on a single
      * node as opposed to multiple nodes.
      */
-    private class MiniFuture extends GridFutureAdapter<IgniteTxEx> {
+    private class MiniFuture extends GridFutureAdapter<IgniteInternalTx> {
         /** */
         private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/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 ec71035..2c55311 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
@@ -278,7 +278,7 @@ public class GridDhtTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> implements
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<IgniteTxEx<K, V>> prepareAsync() {
+    @Override public IgniteInternalFuture<IgniteInternalTx<K, V>> prepareAsync() {
         if (optimistic()) {
             assert isSystemInvalidate();
 
@@ -350,7 +350,7 @@ public class GridDhtTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> implements
      * @param lastBackups IDs of backup nodes receiving last prepare request.
      * @return Future that will be completed when locks are acquired.
      */
-    public IgniteInternalFuture<IgniteTxEx<K, V>> prepareAsync(@Nullable Iterable<IgniteTxEntry<K, V>> reads,
+    public IgniteInternalFuture<IgniteInternalTx<K, V>> prepareAsync(@Nullable Iterable<IgniteTxEntry<K, V>> reads,
         @Nullable Iterable<IgniteTxEntry<K, V>> writes,
         Map<IgniteTxKey<K>, GridCacheVersion> verMap,
         long msgId,
@@ -451,7 +451,7 @@ public class GridDhtTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> implements
 
     /** {@inheritDoc} */
     @SuppressWarnings({"ThrowableInstanceNeverThrown"})
-    @Override public IgniteInternalFuture<IgniteTxEx> commitAsync() {
+    @Override public IgniteInternalFuture<IgniteInternalTx> commitAsync() {
         if (log.isDebugEnabled())
             log.debug("Committing dht local tx: " + this);
 
@@ -488,8 +488,8 @@ public class GridDhtTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> implements
                 }
             }
             else
-                prep.listenAsync(new CI1<IgniteInternalFuture<IgniteTxEx<K, V>>>() {
-                    @Override public void apply(IgniteInternalFuture<IgniteTxEx<K, V>> f) {
+                prep.listenAsync(new CI1<IgniteInternalFuture<IgniteInternalTx<K, V>>>() {
+                    @Override public void apply(IgniteInternalFuture<IgniteInternalTx<K, V>> f) {
                         try {
                             f.get(); // Check for errors of a parent future.
 
@@ -546,7 +546,7 @@ public class GridDhtTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> implements
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<IgniteTxEx> rollbackAsync() {
+    @Override public IgniteInternalFuture<IgniteInternalTx> rollbackAsync() {
         GridDhtTxPrepareFuture<K, V> prepFut = this.prepFut.get();
 
         final GridDhtTxFinishFuture<K, V> fut = new GridDhtTxFinishFuture<>(cctx, this, /*rollback*/false);
@@ -576,8 +576,8 @@ public class GridDhtTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> implements
         else {
             prepFut.complete();
 
-            prepFut.listenAsync(new CI1<IgniteInternalFuture<IgniteTxEx<K, V>>>() {
-                @Override public void apply(IgniteInternalFuture<IgniteTxEx<K, V>> f) {
+            prepFut.listenAsync(new CI1<IgniteInternalFuture<IgniteInternalTx<K, V>>>() {
+                @Override public void apply(IgniteInternalFuture<IgniteInternalTx<K, V>> f) {
                     try {
                         f.get(); // Check for errors of a parent future.
                     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index 72799f1..8f45c7f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -46,8 +46,8 @@ import static org.apache.ignite.internal.managers.communication.GridIoPolicy.*;
 /**
  *
  */
-public final class GridDhtTxPrepareFuture<K, V> extends GridCompoundIdentityFuture<IgniteTxEx<K, V>>
-    implements GridCacheMvccFuture<K, V, IgniteTxEx<K, V>> {
+public final class GridDhtTxPrepareFuture<K, V> extends GridCompoundIdentityFuture<IgniteInternalTx<K, V>>
+    implements GridCacheMvccFuture<K, V, IgniteInternalTx<K, V>> {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -123,12 +123,12 @@ public final class GridDhtTxPrepareFuture<K, V> extends GridCompoundIdentityFutu
      */
     public GridDhtTxPrepareFuture(GridCacheSharedContext<K, V> cctx, final GridDhtTxLocalAdapter<K, V> tx,
         IgniteUuid nearMiniId, Map<IgniteTxKey<K>, GridCacheVersion> dhtVerMap, boolean last, Collection<UUID> lastBackups) {
-        super(cctx.kernalContext(), new IgniteReducer<IgniteTxEx<K, V>, IgniteTxEx<K, V>>() {
-            @Override public boolean collect(IgniteTxEx<K, V> e) {
+        super(cctx.kernalContext(), new IgniteReducer<IgniteInternalTx<K, V>, IgniteInternalTx<K, V>>() {
+            @Override public boolean collect(IgniteInternalTx<K, V> e) {
                 return true;
             }
 
-            @Override public IgniteTxEx<K, V> reduce() {
+            @Override public IgniteInternalTx<K, V> reduce() {
                 // Nothing to aggregate.
                 return tx;
             }
@@ -307,7 +307,7 @@ public final class GridDhtTxPrepareFuture<K, V> extends GridCompoundIdentityFutu
      */
     public void onResult(UUID nodeId, GridDhtTxPrepareResponse<K, V> res) {
         if (!isDone()) {
-            for (IgniteInternalFuture<IgniteTxEx<K, V>> fut : pending()) {
+            for (IgniteInternalFuture<IgniteInternalTx<K, V>> fut : pending()) {
                 if (isMini(fut)) {
                     MiniFuture f = (MiniFuture)fut;
 
@@ -376,7 +376,7 @@ public final class GridDhtTxPrepareFuture<K, V> extends GridCompoundIdentityFutu
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onDone(IgniteTxEx<K, V> tx0, Throwable err) {
+    @Override public boolean onDone(IgniteInternalTx<K, V> tx0, Throwable err) {
         assert err != null || (initialized() && !hasPending()) : "On done called for prepare future that has " +
             "pending mini futures: " + this;
 
@@ -891,7 +891,7 @@ public final class GridDhtTxPrepareFuture<K, V> extends GridCompoundIdentityFutu
      * Mini-future for get operations. Mini-futures are only waiting on a single
      * node as opposed to multiple nodes.
      */
-    private class MiniFuture extends GridFutureAdapter<IgniteTxEx<K, V>> {
+    private class MiniFuture extends GridFutureAdapter<IgniteInternalTx<K, V>> {
         /** */
         private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
index 7d7fb35..235929e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
@@ -174,7 +174,7 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
     /** {@inheritDoc} */
     @SuppressWarnings({"unchecked", "RedundantCast"})
     @Override public IgniteInternalFuture<Object> readThroughAllAsync(Collection<? extends K> keys, boolean reload,
-        IgniteTxEx<K, V> tx, IgnitePredicate<CacheEntry<K, V>>[] filter, @Nullable UUID subjId, String taskName,
+        IgniteInternalTx<K, V> tx, IgnitePredicate<CacheEntry<K, V>>[] filter, @Nullable UUID subjId, String taskName,
         IgniteBiInClosure<K, V> vis) {
         return (IgniteInternalFuture)loadAsync(tx,
             keys,
@@ -272,7 +272,7 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
      * @param expiryPlc Expiry policy.
      * @return Loaded values.
      */
-    public IgniteInternalFuture<Map<K, V>> loadAsync(@Nullable IgniteTxEx tx,
+    public IgniteInternalFuture<Map<K, V>> loadAsync(@Nullable IgniteInternalTx tx,
         @Nullable Collection<? extends K> keys,
         boolean reload,
         boolean forcePrimary,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
index a113a63..2974cf4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
@@ -315,7 +315,7 @@ public class GridNearCacheEntry<K, V> extends GridDistributedCacheEntry<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override protected V readThrough(IgniteTxEx<K, V> tx, K key, boolean reload,
+    @Override protected V readThrough(IgniteInternalTx<K, V> tx, K key, boolean reload,
         IgnitePredicate<CacheEntry<K, V>>[] filter, UUID subjId, String taskName) throws IgniteCheckedException {
         return cctx.near().loadAsync(tx,
             F.asList(key),
@@ -346,7 +346,7 @@ public class GridNearCacheEntry<K, V> extends GridDistributedCacheEntry<K, V> {
      * @throws GridCacheEntryRemovedException If entry was removed.
      */
     @SuppressWarnings({"RedundantTypeArguments"})
-    public boolean loadedValue(@Nullable IgniteTxEx tx, UUID primaryNodeId, V val, byte[] valBytes,
+    public boolean loadedValue(@Nullable IgniteInternalTx tx, UUID primaryNodeId, V val, byte[] valBytes,
         GridCacheVersion ver, GridCacheVersion dhtVer, @Nullable GridCacheVersion expVer, long ttl, long expireTime,
         boolean evt, long topVer, UUID subjId)
         throws IgniteCheckedException, GridCacheEntryRemovedException {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/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 8b7d340..f0fa418 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
@@ -45,8 +45,8 @@ import static org.apache.ignite.internal.processors.cache.GridCacheOperation.*;
 /**
  *
  */
-public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFuture<IgniteTxEx>
-    implements GridCacheFuture<IgniteTxEx> {
+public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFuture<IgniteInternalTx>
+    implements GridCacheFuture<IgniteInternalTx> {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -91,7 +91,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
      * @param commit Commit flag.
      */
     public GridNearTxFinishFuture(GridCacheSharedContext<K, V> cctx, GridNearTxLocal<K, V> tx, boolean commit) {
-        super(cctx.kernalContext(), F.<IgniteTxEx>identityReducer(tx));
+        super(cctx.kernalContext(), F.<IgniteInternalTx>identityReducer(tx));
 
         assert cctx != null;
 
@@ -200,7 +200,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
      */
     public void onResult(UUID nodeId, GridNearTxFinishResponse<K, V> res) {
         if (!isDone())
-            for (IgniteInternalFuture<IgniteTxEx> fut : futures()) {
+            for (IgniteInternalFuture<IgniteInternalTx> fut : futures()) {
                 if (isMini(fut)) {
                     MiniFuture f = (MiniFuture)fut;
 
@@ -214,7 +214,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onDone(IgniteTxEx tx, Throwable err) {
+    @Override public boolean onDone(IgniteInternalTx tx, Throwable err) {
         if ((initialized() || err != null)) {
             if (this.tx.onePhaseCommit() && (this.tx.state() == COMMITTING))
                 this.tx.tmCommit();
@@ -283,7 +283,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
             // Finish local mapping only as we need send commit message to backups.
             for (GridDistributedTxMapping<K, V> m : mappings.values()) {
                 if (m.node().isLocal()) {
-                    IgniteInternalFuture<IgniteTxEx> fut = cctx.tm().txHandler().finishColocatedLocal(commit, tx);
+                    IgniteInternalFuture<IgniteInternalTx> fut = cctx.tm().txHandler().finishColocatedLocal(commit, tx);
 
                     // Add new future.
                     if (fut != null)
@@ -370,7 +370,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
         if (n.isLocal()) {
             req.miniId(IgniteUuid.randomUuid());
 
-            IgniteInternalFuture<IgniteTxEx> fut = cctx.tm().txHandler().finish(n.id(), tx, req);
+            IgniteInternalFuture<IgniteInternalTx> fut = cctx.tm().txHandler().finish(n.id(), tx, req);
 
             // Add new future.
             if (fut != null)
@@ -415,7 +415,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
      * Mini-future for get operations. Mini-futures are only waiting on a single
      * node as opposed to multiple nodes.
      */
-    private class MiniFuture extends GridFutureAdapter<IgniteTxEx> {
+    private class MiniFuture extends GridFutureAdapter<IgniteInternalTx> {
         /** */
         private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/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 e39d67f..f8a482b 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
@@ -62,7 +62,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
         new ConcurrentHashMap8<>();
 
     /** Future. */
-    private final AtomicReference<IgniteInternalFuture<IgniteTxEx<K, V>>> prepFut =
+    private final AtomicReference<IgniteInternalFuture<IgniteInternalTx<K, V>>> prepFut =
         new AtomicReference<>();
 
     /** */
@@ -542,7 +542,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
 
     /** {@inheritDoc} */
     @Override public boolean onOwnerChanged(GridCacheEntryEx<K, V> entry, GridCacheMvccCandidate<K> owner) {
-        GridCacheMvccFuture<K, V, IgniteTxEx<K, V>> fut = (GridCacheMvccFuture<K, V, IgniteTxEx<K, V>>)prepFut
+        GridCacheMvccFuture<K, V, IgniteInternalTx<K, V>> fut = (GridCacheMvccFuture<K, V, IgniteInternalTx<K, V>>)prepFut
             .get();
 
         return fut != null && fut.onOwnerChanged(entry, owner);
@@ -695,8 +695,8 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<IgniteTxEx<K, V>> prepareAsync() {
-        IgniteInternalFuture<IgniteTxEx<K, V>> fut = prepFut.get();
+    @Override public IgniteInternalFuture<IgniteInternalTx<K, V>> prepareAsync() {
+        IgniteInternalFuture<IgniteInternalTx<K, V>> fut = prepFut.get();
 
         if (fut == null) {
             // Future must be created before any exception can be thrown.
@@ -763,7 +763,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings({"ThrowableInstanceNeverThrown"})
-    @Override public IgniteInternalFuture<IgniteTxEx> commitAsync() {
+    @Override public IgniteInternalFuture<IgniteInternalTx> commitAsync() {
         if (log.isDebugEnabled())
             log.debug("Committing near local tx: " + this);
 
@@ -776,10 +776,10 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
 
         cctx.mvcc().addFuture(fut);
 
-        IgniteInternalFuture<IgniteTxEx<K, V>> prepareFut = prepFut.get();
+        IgniteInternalFuture<IgniteInternalTx<K, V>> prepareFut = prepFut.get();
 
-        prepareFut.listenAsync(new CI1<IgniteInternalFuture<IgniteTxEx<K, V>>>() {
-            @Override public void apply(IgniteInternalFuture<IgniteTxEx<K, V>> f) {
+        prepareFut.listenAsync(new CI1<IgniteInternalFuture<IgniteInternalTx<K, V>>>() {
+            @Override public void apply(IgniteInternalFuture<IgniteInternalTx<K, V>> f) {
                 GridNearTxFinishFuture<K, V> fut0 = commitFut.get();
 
                 try {
@@ -809,7 +809,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<IgniteTxEx> rollbackAsync() {
+    @Override public IgniteInternalFuture<IgniteInternalTx> rollbackAsync() {
         if (log.isDebugEnabled())
             log.debug("Rolling back near tx: " + this);
 
@@ -823,7 +823,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
 
         cctx.mvcc().addFuture(fut);
 
-        IgniteInternalFuture<IgniteTxEx<K, V>> prepFut = this.prepFut.get();
+        IgniteInternalFuture<IgniteInternalTx<K, V>> prepFut = this.prepFut.get();
 
         if (prepFut == null || prepFut.isDone()) {
             try {
@@ -847,8 +847,8 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
             }
         }
         else {
-            prepFut.listenAsync(new CI1<IgniteInternalFuture<IgniteTxEx<K, V>>>() {
-                @Override public void apply(IgniteInternalFuture<IgniteTxEx<K, V>> f) {
+            prepFut.listenAsync(new CI1<IgniteInternalFuture<IgniteInternalTx<K, V>>>() {
+                @Override public void apply(IgniteInternalFuture<IgniteInternalTx<K, V>> f) {
                     try {
                         // Check for errors in prepare future.
                         f.get();
@@ -891,7 +891,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
      * @return Future that will be completed when locks are acquired.
      */
     @SuppressWarnings("TypeMayBeWeakened")
-    public IgniteInternalFuture<IgniteTxEx<K, V>> prepareAsyncLocal(@Nullable Collection<IgniteTxEntry<K, V>> reads,
+    public IgniteInternalFuture<IgniteInternalTx<K, V>> prepareAsyncLocal(@Nullable Collection<IgniteTxEntry<K, V>> reads,
         @Nullable Collection<IgniteTxEntry<K, V>> writes, Map<UUID, Collection<UUID>> txNodes, boolean last,
         Collection<UUID> lastBackups) {
         assert optimistic();
@@ -957,7 +957,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
      *
      * @return Commit future.
      */
-    public IgniteInternalFuture<IgniteTxEx> commitAsyncLocal() {
+    public IgniteInternalFuture<IgniteInternalTx> commitAsyncLocal() {
         if (log.isDebugEnabled())
             log.debug("Committing colocated tx locally: " + this);
 
@@ -965,14 +965,14 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
         if (pessimistic())
             prepareAsync();
 
-        IgniteInternalFuture<IgniteTxEx<K, V>> prep = prepFut.get();
+        IgniteInternalFuture<IgniteInternalTx<K, V>> prep = prepFut.get();
 
         // Do not create finish future if there are no remote nodes.
         if (F.isEmpty(dhtMap) && F.isEmpty(nearMap)) {
             if (prep != null)
-                return (IgniteInternalFuture<IgniteTxEx>)(IgniteInternalFuture)prep;
+                return (IgniteInternalFuture<IgniteInternalTx>)(IgniteInternalFuture)prep;
 
-            return new GridFinishedFuture<IgniteTxEx>(cctx.kernalContext(), this);
+            return new GridFinishedFuture<IgniteInternalTx>(cctx.kernalContext(), this);
         }
 
         final GridDhtTxFinishFuture<K, V> fut = new GridDhtTxFinishFuture<>(cctx, this, /*commit*/true);
@@ -1001,8 +1001,8 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
             }
         }
         else
-            prep.listenAsync(new CI1<IgniteInternalFuture<IgniteTxEx<K, V>>>() {
-                @Override public void apply(IgniteInternalFuture<IgniteTxEx<K, V>> f) {
+            prep.listenAsync(new CI1<IgniteInternalFuture<IgniteInternalTx<K, V>>>() {
+                @Override public void apply(IgniteInternalFuture<IgniteInternalTx<K, V>> f) {
                     try {
                         f.get(); // Check for errors of a parent future.
 
@@ -1030,7 +1030,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
      *
      * @return Commit future.
      */
-    public IgniteInternalFuture<IgniteTxEx> rollbackAsyncLocal() {
+    public IgniteInternalFuture<IgniteInternalTx> rollbackAsyncLocal() {
         if (log.isDebugEnabled())
             log.debug("Rolling back colocated tx locally: " + this);
 
@@ -1038,7 +1038,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
 
         cctx.mvcc().addFuture(fut);
 
-        IgniteInternalFuture<IgniteTxEx<K, V>> prep = prepFut.get();
+        IgniteInternalFuture<IgniteInternalTx<K, V>> prep = prepFut.get();
 
         if (prep == null || prep.isDone()) {
             try {
@@ -1054,8 +1054,8 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
             fut.finish();
         }
         else
-            prep.listenAsync(new CI1<IgniteInternalFuture<IgniteTxEx<K, V>>>() {
-                @Override public void apply(IgniteInternalFuture<IgniteTxEx<K, V>> f) {
+            prep.listenAsync(new CI1<IgniteInternalFuture<IgniteInternalTx<K, V>>>() {
+                @Override public void apply(IgniteInternalFuture<IgniteInternalTx<K, V>> f) {
                     try {
                         f.get(); // Check for errors of a parent future.
                     }
@@ -1245,13 +1245,13 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
     /**
      *
      */
-    private static class PessimisticPrepareFuture<K, V> extends GridFutureAdapter<IgniteTxEx<K, V>> {
+    private static class PessimisticPrepareFuture<K, V> extends GridFutureAdapter<IgniteInternalTx<K, V>> {
         /** */
         private static final long serialVersionUID = 0L;
 
         /** Transaction. */
         @GridToStringExclude
-        private IgniteTxEx<K, V> tx;
+        private IgniteInternalTx<K, V> tx;
 
         /**
          * Empty constructor required by {@link Externalizable}.
@@ -1264,7 +1264,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
          * @param ctx Kernal context.
          * @param tx Transaction.
          */
-        private PessimisticPrepareFuture(GridKernalContext ctx, IgniteTxEx<K, V> tx) {
+        private PessimisticPrepareFuture(GridKernalContext ctx, IgniteInternalTx<K, V> tx) {
             super(ctx);
             this.tx = tx;
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
index 134a8ff..0898003 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
@@ -48,8 +48,8 @@ import static org.apache.ignite.internal.processors.cache.GridCacheOperation.*;
 /**
  *
  */
-public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFuture<IgniteTxEx<K, V>>
-    implements GridCacheMvccFuture<K, V, IgniteTxEx<K, V>> {
+public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFuture<IgniteInternalTx<K, V>>
+    implements GridCacheMvccFuture<K, V, IgniteInternalTx<K, V>> {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -91,12 +91,12 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
      * @param tx Transaction.
      */
     public GridNearTxPrepareFuture(GridCacheSharedContext<K, V> cctx, final GridNearTxLocal<K, V> tx) {
-        super(cctx.kernalContext(), new IgniteReducer<IgniteTxEx<K, V>, IgniteTxEx<K, V>>() {
-            @Override public boolean collect(IgniteTxEx<K, V> e) {
+        super(cctx.kernalContext(), new IgniteReducer<IgniteInternalTx<K, V>, IgniteInternalTx<K, V>>() {
+            @Override public boolean collect(IgniteInternalTx<K, V> e) {
                 return true;
             }
 
-            @Override public IgniteTxEx<K, V> reduce() {
+            @Override public IgniteInternalTx<K, V> reduce() {
                 // Nothing to aggregate.
                 return tx;
             }
@@ -263,7 +263,7 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
      */
     public void onResult(UUID nodeId, GridNearTxPrepareResponse<K, V> res) {
         if (!isDone()) {
-            for (IgniteInternalFuture<IgniteTxEx<K, V>> fut : pending()) {
+            for (IgniteInternalFuture<IgniteInternalTx<K, V>> fut : pending()) {
                 if (isMini(fut)) {
                     MiniFuture f = (MiniFuture)fut;
 
@@ -278,7 +278,7 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onDone(IgniteTxEx<K, V> t, Throwable err) {
+    @Override public boolean onDone(IgniteInternalTx<K, V> t, Throwable err) {
         // If locks were not acquired yet, delay completion.
         if (isDone() || (err == null && !checkLocks()))
             return false;
@@ -601,14 +601,14 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
             // At this point, if any new node joined, then it is
             // waiting for this transaction to complete, so
             // partition reassignments are not possible here.
-            IgniteInternalFuture<IgniteTxEx<K, V>> fut = cctx.tm().txHandler().prepareTx(n.id(), tx, req);
+            IgniteInternalFuture<IgniteInternalTx<K, V>> fut = cctx.tm().txHandler().prepareTx(n.id(), tx, req);
 
             // Add new future.
             add(new GridEmbeddedFuture<>(
                 cctx.kernalContext(),
                 fut,
-                new C2<IgniteTxEx<K, V>, Exception, IgniteTxEx<K, V>>() {
-                    @Override public IgniteTxEx<K, V> apply(IgniteTxEx<K, V> t, Exception ex) {
+                new C2<IgniteInternalTx<K, V>, Exception, IgniteInternalTx<K, V>>() {
+                    @Override public IgniteInternalTx<K, V> apply(IgniteInternalTx<K, V> t, Exception ex) {
                         if (ex != null) {
                             onError(n.id(), mappings, ex);
 
@@ -737,7 +737,7 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
      * Mini-future for get operations. Mini-futures are only waiting on a single
      * node as opposed to multiple nodes.
      */
-    private class MiniFuture extends GridFutureAdapter<IgniteTxEx<K, V>> {
+    private class MiniFuture extends GridFutureAdapter<IgniteInternalTx<K, V>> {
         /** */
         private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
index 8aecfe0..5fd686a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
@@ -90,7 +90,7 @@ public class GridNearTxPrepareRequest<K, V> extends GridDistributedTxPrepareRequ
     public GridNearTxPrepareRequest(
         IgniteUuid futId,
         long topVer,
-        IgniteTxEx<K, V> tx,
+        IgniteInternalTx<K, V> tx,
         Collection<IgniteTxEntry<K, V>> reads,
         Collection<IgniteTxEntry<K, V>> writes,
         IgniteTxKey grpLockKey,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
index 0144617..6c51860 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
@@ -180,7 +180,7 @@ public class GridLocalCacheEntry<K, V> extends GridCacheMapEntry<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean tmLock(IgniteTxEx<K, V> tx, long timeout) throws GridCacheEntryRemovedException {
+    @Override public boolean tmLock(IgniteInternalTx<K, V> tx, long timeout) throws GridCacheEntryRemovedException {
         GridCacheMvccCandidate<K> cand = addLocal(
             tx.threadId(),
             tx.xidVersion(),
@@ -276,7 +276,7 @@ public class GridLocalCacheEntry<K, V> extends GridCacheMapEntry<K, V> {
      *
      * @param tx Transaction to unlock.
      */
-    @Override public void txUnlock(IgniteTxEx<K, V> tx) throws GridCacheEntryRemovedException {
+    @Override public void txUnlock(IgniteInternalTx<K, V> tx) throws GridCacheEntryRemovedException {
         removeLock(tx.xidVersion());
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTx.java
index e53c98b..8a0f0b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTx.java
@@ -107,11 +107,11 @@ class GridLocalTx<K, V> extends IgniteTxLocalAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<IgniteTxEx<K, V>> prepareAsync() {
+    @Override public IgniteInternalFuture<IgniteInternalTx<K, V>> prepareAsync() {
         try {
             prepare();
 
-            return new GridFinishedFuture<IgniteTxEx<K, V>>(cctx.kernalContext(), this);
+            return new GridFinishedFuture<IgniteInternalTx<K, V>>(cctx.kernalContext(), this);
         }
         catch (IgniteCheckedException e) {
             return new GridFinishedFuture<>(cctx.kernalContext(), e);
@@ -146,7 +146,7 @@ class GridLocalTx<K, V> extends IgniteTxLocalAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings( {"unchecked", "RedundantCast"})
-    @Override public IgniteInternalFuture<IgniteTxEx> commitAsync() {
+    @Override public IgniteInternalFuture<IgniteInternalTx> commitAsync() {
         try {
             prepare();
         }
@@ -177,7 +177,7 @@ class GridLocalTx<K, V> extends IgniteTxLocalAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<IgniteTxEx> rollbackAsync() {
+    @Override public IgniteInternalFuture<IgniteInternalTx> rollbackAsync() {
         try {
             state(ROLLING_BACK);
 
@@ -185,7 +185,7 @@ class GridLocalTx<K, V> extends IgniteTxLocalAdapter<K, V> {
 
             state(ROLLED_BACK);
 
-            return new GridFinishedFuture<IgniteTxEx>(cctx.kernalContext(), this);
+            return new GridFinishedFuture<IgniteInternalTx>(cctx.kernalContext(), this);
         }
         catch (IgniteCheckedException e) {
             return new GridFinishedFuture<>(cctx.kernalContext(), e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTxFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTxFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTxFuture.java
index 552e7c0..6784c1b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTxFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTxFuture.java
@@ -37,8 +37,8 @@ import static org.apache.ignite.transactions.IgniteTxState.*;
 /**
  * Replicated cache transaction future.
  */
-final class GridLocalTxFuture<K, V> extends GridFutureAdapter<IgniteTxEx<K, V>>
-    implements GridCacheMvccFuture<K, V, IgniteTxEx<K, V>> {
+final class GridLocalTxFuture<K, V> extends GridFutureAdapter<IgniteInternalTx<K, V>>
+    implements GridCacheMvccFuture<K, V, IgniteInternalTx<K, V>> {
     /** */
     private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
new file mode 100644
index 0000000..be83e25
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
@@ -0,0 +1,706 @@
+/*
+ * 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.transactions;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.cache.version.*;
+import org.apache.ignite.internal.transactions.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.transactions.*;
+import org.apache.ignite.internal.processors.timeout.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Transaction managed by cache ({@code 'Ex'} stands for external).
+ */
+public interface IgniteInternalTx<K, V> extends AutoCloseable, GridTimeoutObject {
+    /**
+     *
+     */
+    @SuppressWarnings("PublicInnerClass")
+    public enum FinalizationStatus {
+        /** Transaction was not finalized yet. */
+        NONE,
+
+        /** Transaction is being finalized by user. */
+        USER_FINISH,
+
+        /** Recovery request is received, user finish requests should be ignored. */
+        RECOVERY_WAIT,
+
+        /** Transaction is being finalized by recovery procedure. */
+        RECOVERY_FINISH
+    }
+
+    /**
+     * Gets unique identifier for this transaction.
+     *
+     * @return Transaction UID.
+     */
+    public IgniteUuid xid();
+
+    /**
+     * ID of the node on which this transaction started.
+     *
+     * @return Originating node ID.
+     */
+    public UUID nodeId();
+
+    /**
+     * ID of the thread in which this transaction started.
+     *
+     * @return Thread ID.
+     */
+    public long threadId();
+
+    /**
+     * Start time of this transaction.
+     *
+     * @return Start time of this transaction on this node.
+     */
+    public long startTime();
+
+    /**
+     * Cache transaction isolation level.
+     *
+     * @return Isolation level.
+     */
+    public IgniteTxIsolation isolation();
+
+    /**
+     * Cache transaction concurrency mode.
+     *
+     * @return Concurrency mode.
+     */
+    public IgniteTxConcurrency concurrency();
+
+    /**
+     * Flag indicating whether transaction was started automatically by the
+     * system or not. System will start transactions implicitly whenever
+     * any cache {@code put(..)} or {@code remove(..)} operation is invoked
+     * outside of transaction.
+     *
+     * @return {@code True} if transaction was started implicitly.
+     */
+    public boolean implicit();
+
+    /**
+     * Get invalidation flag for this transaction. If set to {@code true}, then
+     * remote values will be {@code invalidated} (set to {@code null}) instead
+     * of updated.
+     * <p>
+     * Invalidation messages don't carry new values, so they are a lot lighter
+     * than update messages. However, when a value is accessed on a node after
+     * it's been invalidated, it must be loaded from persistent store.
+     *
+     * @return Invalidation flag.
+     */
+    public boolean isInvalidate();
+
+    /**
+     * Gets current transaction state value.
+     *
+     * @return Current transaction state.
+     */
+    public IgniteTxState state();
+
+    /**
+     * Gets timeout value in milliseconds for this transaction. If transaction times
+     * out prior to it's completion, {@link org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException} will be thrown.
+     *
+     * @return Transaction timeout value.
+     */
+    public long timeout();
+
+    /**
+     * Sets transaction timeout value. This value can be set only before a first operation
+     * on transaction has been performed.
+     *
+     * @param timeout Transaction timeout value.
+     * @return Previous timeout.
+     */
+    public long timeout(long timeout);
+
+    /**
+     * Modify the transaction associated with the current thread such that the
+     * only possible outcome of the transaction is to roll back the
+     * transaction.
+     *
+     * @return {@code True} if rollback-only flag was set as a result of this operation,
+     *      {@code false} if it was already set prior to this call or could not be set
+     *      because transaction is already finishing up committing or rolling back.
+     */
+    public boolean setRollbackOnly();
+
+    /**
+     * If transaction was marked as rollback-only.
+     *
+     * @return {@code True} if transaction can only be rolled back.
+     */
+    public boolean isRollbackOnly();
+
+    /**
+     * Commits this transaction by initiating {@code two-phase-commit} process.
+     *
+     * @throws IgniteCheckedException If commit failed.
+     */
+    @IgniteAsyncSupported
+    public void commit() throws IgniteCheckedException;
+
+    /**
+     * Ends the transaction. Transaction will be rolled back if it has not been committed.
+     *
+     * @throws IgniteCheckedException If transaction could not be gracefully ended.
+     */
+    @Override public void close() throws IgniteCheckedException;
+
+    /**
+     * Rolls back this transaction.
+     *
+     * @throws IgniteCheckedException If rollback failed.
+     */
+    @IgniteAsyncSupported
+    public void rollback() throws IgniteCheckedException;
+
+    /**
+     * Removes metadata by name.
+     *
+     * @param name Name of the metadata to remove.
+     * @param <T> Type of the value.
+     * @return Value of removed metadata or {@code null}.
+     */
+    @Nullable public <T> T removeMeta(String name);
+
+    /**
+     * Gets metadata by name.
+     *
+     * @param name Metadata name.
+     * @param <T> Type of the value.
+     * @return Metadata value or {@code null}.
+     */
+    @Nullable public <T> T meta(String name);
+
+    /**
+     * Adds a new metadata.
+     *
+     * @param name Metadata name.
+     * @param val Metadata value.
+     * @param <T> Type of the value.
+     * @return Metadata previously associated with given name, or
+     *      {@code null} if there was none.
+     */
+    @Nullable public <T> T addMeta(String name, T val);
+
+    /**
+     * @return Size of the transaction.
+     */
+    public int size();
+
+    /**
+     * @return {@code True} if transaction is allowed to use store.
+     */
+    public boolean storeEnabled();
+
+    /**
+     * @return {@code True} if transaction is allowed to use store and transactions spans one or more caches with
+     *      store enabled.
+     */
+    public boolean storeUsed();
+
+    /**
+     * Checks if this is system cache transaction. System transactions are isolated from user transactions
+     * because some of the public API methods may be invoked inside user transactions and internally start
+     * system cache transactions.
+     *
+     * @return {@code True} if transaction is started for system cache.
+     */
+    public boolean system();
+
+    /**
+     * @return Last recorded topology version.
+     */
+    public long topologyVersion();
+
+    /**
+     * @return Flag indicating whether transaction is implicit with only one key.
+     */
+    public boolean implicitSingle();
+
+    /**
+     * @return Collection of cache IDs involved in this transaction.
+     */
+    public Collection<Integer> activeCacheIds();
+
+    /**
+     * Attempts to set topology version and returns the current value.
+     * If topology version was previously set, then it's value will
+     * be returned (but not updated).
+     *
+     * @param topVer Topology version.
+     * @return Recorded topology version.
+     */
+    public long topologyVersion(long topVer);
+
+    /**
+     * @return {@code True} if transaction is empty.
+     */
+    public boolean empty();
+
+    /**
+     * @return {@code True} if transaction group-locked.
+     */
+    public boolean groupLock();
+
+    /**
+     * @return Group lock key if {@link #groupLock()} is {@code true}.
+     */
+    @Nullable public IgniteTxKey groupLockKey();
+
+    /**
+     * @return {@code True} if preparing flag was set with this call.
+     */
+    public boolean markPreparing();
+
+    /**
+     * @param status Finalization status to set.
+     * @return {@code True} if could mark was set.
+     */
+    public boolean markFinalizing(FinalizationStatus status);
+
+    /**
+     * @param cacheCtx Cache context.
+     * @param part Invalid partition.
+     */
+    public void addInvalidPartition(GridCacheContext<K, V> cacheCtx, int part);
+
+    /**
+     * @return Invalid partitions.
+     */
+    public Set<Integer> invalidPartitions();
+
+    /**
+     * Gets owned version for near remote transaction.
+     *
+     * @param key Key to get version for.
+     * @return Owned version, if any.
+     */
+    @Nullable public GridCacheVersion ownedVersion(IgniteTxKey<K> key);
+
+    /**
+     * Gets ID of additional node involved. For example, in DHT case, other node is
+     * near node ID.
+     *
+     * @return Parent node IDs.
+     */
+    @Nullable public UUID otherNodeId();
+
+    /**
+     * @return Event node ID.
+     */
+    public UUID eventNodeId();
+
+    /**
+     * Gets node ID which directly started this transaction. In case of DHT local transaction it will be
+     * near node ID, in case of DHT remote transaction it will be primary node ID, in case of replicated remote
+     * transaction it will be starter node ID.
+     *
+     * @return Originating node ID.
+     */
+    public UUID originatingNodeId();
+
+    /**
+     * @return Master node IDs.
+     */
+    public Collection<UUID> masterNodeIds();
+
+    /**
+     * @return Near transaction ID.
+     */
+    @Nullable public GridCacheVersion nearXidVersion();
+
+    /**
+     * @return Transaction nodes mapping (primary node -> related backup nodes).
+     */
+    @Nullable public Map<UUID, Collection<UUID>> transactionNodes();
+
+    /**
+     * @param entry Entry to check.
+     * @return {@code True} if lock is owned.
+     * @throws GridCacheEntryRemovedException If entry has been removed.
+     */
+    public boolean ownsLock(GridCacheEntryEx<K, V> entry) throws GridCacheEntryRemovedException;
+
+    /**
+     * @param entry Entry to check.
+     * @return {@code True} if lock is owned.
+     */
+    public boolean ownsLockUnsafe(GridCacheEntryEx<K, V> entry);
+
+    /**
+     * For Partitioned caches, this flag is {@code false} for remote DHT and remote NEAR
+     * transactions because serializability of transaction is enforced on primary node. All
+     * other transaction types must enforce it.
+     *
+     * @return Enforce serializable flag.
+     */
+    public boolean enforceSerializable();
+
+    /**
+     * @return {@code True} if near transaction.
+     */
+    public boolean near();
+
+    /**
+     * @return {@code True} if DHT transaction.
+     */
+    public boolean dht();
+
+    /**
+     * @return {@code True} if dht colocated transaction.
+     */
+    public boolean colocated();
+
+    /**
+     * @return {@code True} if transaction is local, {@code false} if it's remote.
+     */
+    public boolean local();
+
+    /**
+     * @return {@code True} if transaction is replicated.
+     */
+    public boolean replicated();
+
+    /**
+     * @return Subject ID initiated this transaction.
+     */
+    public UUID subjectId();
+
+    /**
+     * Task name hash in case if transaction was initiated within task execution.
+     *
+     * @return Task name hash.
+     */
+    public int taskNameHash();
+
+    /**
+     * @return {@code True} if transaction is user transaction, which means:
+     * <ul>
+     *     <li>Explicit</li>
+     *     <li>Local</li>
+     *     <li>Not DHT</li>
+     * </ul>
+     */
+    public boolean user();
+
+    /**
+     * @return {@code True} if transaction is configured with synchronous commit flag.
+     */
+    public boolean syncCommit();
+
+    /**
+     * @return {@code True} if transaction is configured with synchronous rollback flag.
+     */
+    public boolean syncRollback();
+
+    /**
+     * @param key Key to check.
+     * @return {@code True} if key is present.
+     */
+    public boolean hasWriteKey(IgniteTxKey<K> key);
+
+    /**
+     * @return Read set.
+     */
+    public Set<IgniteTxKey<K>> readSet();
+
+    /**
+     * @return Write set.
+     */
+    public Set<IgniteTxKey<K>> writeSet();
+
+    /**
+     * @return All transaction entries.
+     */
+    public Collection<IgniteTxEntry<K, V>> allEntries();
+
+    /**
+     * @return Write entries.
+     */
+    public Collection<IgniteTxEntry<K, V>> writeEntries();
+
+    /**
+     * @return Read entries.
+     */
+    public Collection<IgniteTxEntry<K, V>> readEntries();
+
+    /**
+     * @return Transaction write map.
+     */
+    public Map<IgniteTxKey<K>, IgniteTxEntry<K, V>> writeMap();
+
+    /**
+     * @return Transaction read map.
+     */
+    public Map<IgniteTxKey<K>, IgniteTxEntry<K, V>> readMap();
+
+    /**
+     * Gets pessimistic recovery writes, i.e. values that have never been sent to remote nodes with lock requests.
+     *
+     * @return Collection of recovery writes.
+     */
+    public Collection<IgniteTxEntry<K, V>> recoveryWrites();
+
+    /**
+     * Gets a list of entries that needs to be locked on the next step of prepare stage of
+     * optimistic transaction.
+     *
+     * @return List of tx entries for optimistic locking.
+     */
+    public Collection<IgniteTxEntry<K, V>> optimisticLockEntries();
+
+    /**
+     * Seals transaction for updates.
+     */
+    public void seal();
+
+    /**
+     * @param key Key for the entry.
+     * @return Entry for the key (either from write set or read set).
+     */
+    @Nullable public IgniteTxEntry<K, V> entry(IgniteTxKey<K> key);
+
+    /**
+     * @param ctx Cache context.
+     * @param failFast Fail-fast flag.
+     * @param key Key to look up.
+     * @param filter Filter to check.
+     * @return Current value for the key within transaction.
+     * @throws GridCacheFilterFailedException If filter failed and failFast is {@code true}.
+     */
+     @Nullable public GridTuple<V> peek(
+         GridCacheContext<K, V> ctx,
+         boolean failFast,
+         K key,
+         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) throws GridCacheFilterFailedException;
+
+    /**
+     * @return Start version.
+     */
+    public GridCacheVersion startVersion();
+
+    /**
+     * @return Transaction version.
+     */
+    public GridCacheVersion xidVersion();
+
+    /**
+     * @return Version created at commit time.
+     */
+    public GridCacheVersion commitVersion();
+
+    /**
+     * @param commitVer Commit version.
+     * @return {@code True} if version was set.
+     */
+    public boolean commitVersion(GridCacheVersion commitVer);
+
+    /**
+     * @return End version (a.k.a. <tt>'tnc'</tt> or <tt>'transaction number counter'</tt>)
+     *      assigned to this transaction at the end of write phase.
+     */
+    public GridCacheVersion endVersion();
+
+    /**
+     * Prepare state.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    public void prepare() throws IgniteCheckedException;
+
+    /**
+     * Prepare stage.
+     *
+     * @return Future for prepare step.
+     */
+    public IgniteInternalFuture<IgniteInternalTx<K, V>> prepareAsync();
+
+    /**
+     * @param endVer End version (a.k.a. <tt>'tnc'</tt> or <tt>'transaction number counter'</tt>)
+     *      assigned to this transaction at the end of write phase.
+     */
+    public void endVersion(GridCacheVersion endVer);
+
+    /**
+     * @return Transaction write version. For all transactions except DHT transactions, will be equal to
+     *      {@link #xidVersion()}.
+     */
+    public GridCacheVersion writeVersion();
+
+    /**
+     * Sets write version.
+     *
+     * @param ver Write version.
+     */
+    public void writeVersion(GridCacheVersion ver);
+
+    /**
+     * @return Future for transaction completion.
+     */
+    public IgniteInternalFuture<IgniteInternalTx> finishFuture();
+
+    /**
+     * @param state Transaction state.
+     * @return {@code True} if transition was valid, {@code false} otherwise.
+     */
+    public boolean state(IgniteTxState state);
+
+    /**
+     * @param invalidate Invalidate flag.
+     */
+    public void invalidate(boolean invalidate);
+
+    /**
+     * @param sysInvalidate System invalidate flag.
+     */
+    public void systemInvalidate(boolean sysInvalidate);
+
+    /**
+     * @return System invalidate flag.
+     */
+    public boolean isSystemInvalidate();
+
+    /**
+     * Asynchronously rollback this transaction.
+     *
+     * @return Rollback future.
+     */
+    public IgniteInternalFuture<IgniteInternalTx> rollbackAsync();
+
+    /**
+     * Asynchronously commits this transaction by initiating {@code two-phase-commit} process.
+     *
+     * @return Future for commit operation.
+     */
+    public IgniteInternalFuture<IgniteInternalTx> commitAsync();
+
+    /**
+     * Callback invoked whenever there is a lock that has been acquired
+     * by this transaction for any of the participating entries.
+     *
+     * @param entry Cache entry.
+     * @param owner Lock candidate that won ownership of the lock.
+     * @return {@code True} if transaction cared about notification.
+     */
+    public boolean onOwnerChanged(GridCacheEntryEx<K, V> entry, GridCacheMvccCandidate<K> owner);
+
+    /**
+     * @return {@code True} if transaction timed out.
+     */
+    public boolean timedOut();
+
+    /**
+     * @return {@code True} if transaction had completed successfully or unsuccessfully.
+     */
+    public boolean done();
+
+    /**
+     * @return {@code True} for OPTIMISTIC transactions.
+     */
+    public boolean optimistic();
+
+    /**
+     * @return {@code True} for PESSIMISTIC transactions.
+     */
+    public boolean pessimistic();
+
+    /**
+     * @return {@code True} if read-committed.
+     */
+    public boolean readCommitted();
+
+    /**
+     * @return {@code True} if repeatable-read.
+     */
+    public boolean repeatableRead();
+
+    /**
+     * @return {@code True} if serializable.
+     */
+    public boolean serializable();
+
+    /**
+     * Checks whether given key has been removed within transaction.
+     *
+     * @param key Key to check.
+     * @return {@code True} if key has been removed.
+     */
+    public boolean removed(IgniteTxKey<K> key);
+
+    /**
+     * Gets allowed remaining time for this transaction.
+     *
+     * @return Remaining time.
+     * @throws org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException If transaction timed out.
+     */
+    public long remainingTime() throws IgniteTxTimeoutCheckedException;
+
+    /**
+     * @return Alternate transaction versions.
+     */
+    public Collection<GridCacheVersion> alternateVersions();
+
+    /**
+     * @return {@code True} if transaction needs completed versions for processing.
+     */
+    public boolean needsCompletedVersions();
+
+    /**
+     * @param base Base for committed versions.
+     * @param committed Committed transactions relative to base.
+     * @param rolledback Rolled back transactions relative to base.
+     */
+    public void completedVersions(GridCacheVersion base, Collection<GridCacheVersion> committed,
+        Collection<GridCacheVersion> rolledback);
+
+    /**
+     * @return {@code True} if transaction has at least one internal entry.
+     */
+    public boolean internal();
+
+    /**
+     * @return {@code True} if transaction is a one-phase-commit transaction.
+     */
+    public boolean onePhaseCommit();
+
+    /**
+     * @return {@code True} if transaction has transform entries. This flag will be only set for local
+     *      transactions.
+     */
+    public boolean hasTransforms();
+
+    /**
+     * @return Public API proxy.
+     */
+    public IgniteTxProxy proxy();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java
index 7c39317..f817ee9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java
@@ -92,7 +92,7 @@ public class IgniteTransactionsImpl<K, V> implements IgniteTransactionsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteTxEx txStartEx(
+    @Override public IgniteInternalTx txStartEx(
         GridCacheContext ctx,
         IgniteTxConcurrency concurrency,
         IgniteTxIsolation isolation,
@@ -112,7 +112,7 @@ public class IgniteTransactionsImpl<K, V> implements IgniteTransactionsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteTxEx txStartEx(
+    @Override public IgniteInternalTx txStartEx(
         GridCacheContext ctx,
         IgniteTxConcurrency concurrency,
         IgniteTxIsolation isolation)
@@ -130,7 +130,7 @@ public class IgniteTransactionsImpl<K, V> implements IgniteTransactionsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteTxEx txStartAffinity(GridCacheContext ctx,
+    @Override public IgniteInternalTx txStartAffinity(GridCacheContext ctx,
         Object affinityKey,
         IgniteTxConcurrency concurrency,
         IgniteTxIsolation isolation,
@@ -149,7 +149,7 @@ public class IgniteTransactionsImpl<K, V> implements IgniteTransactionsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteTxEx txStartPartitionEx(GridCacheContext ctx,
+    @Override public IgniteInternalTx txStartPartitionEx(GridCacheContext ctx,
         int partId,
         IgniteTxConcurrency concurrency,
         IgniteTxIsolation isolation,
@@ -194,7 +194,7 @@ public class IgniteTransactionsImpl<K, V> implements IgniteTransactionsEx {
      * @param sys System flag.
      * @return Transaction.
      */
-    private IgniteTxEx txStart0(IgniteTxConcurrency concurrency, IgniteTxIsolation isolation,
+    private IgniteInternalTx txStart0(IgniteTxConcurrency concurrency, IgniteTxIsolation isolation,
         long timeout, int txSize, boolean sys) {
         TransactionsConfiguration cfg = cctx.gridConfig().getTransactionsConfiguration();
 
@@ -202,7 +202,7 @@ public class IgniteTransactionsImpl<K, V> implements IgniteTransactionsEx {
             throw new IllegalArgumentException("SERIALIZABLE isolation level is disabled (to enable change " +
                 "'txSerializableEnabled' configuration property)");
 
-        IgniteTxEx<K, V> tx = (IgniteTxEx<K, V>)cctx.tm().userTx();
+        IgniteInternalTx<K, V> tx = (IgniteInternalTx<K, V>)cctx.tm().userTx();
 
         if (tx != null)
             throw new IllegalStateException("Failed to start new transaction " +
@@ -294,10 +294,10 @@ public class IgniteTransactionsImpl<K, V> implements IgniteTransactionsEx {
      * @throws IgniteCheckedException In case of error.
      */
     @SuppressWarnings("unchecked")
-    private IgniteTxEx txStartGroupLock(GridCacheContext ctx, Object grpLockKey, IgniteTxConcurrency concurrency,
+    private IgniteInternalTx txStartGroupLock(GridCacheContext ctx, Object grpLockKey, IgniteTxConcurrency concurrency,
         IgniteTxIsolation isolation, boolean partLock, long timeout, int txSize, boolean sys)
         throws IllegalStateException, IgniteCheckedException {
-        IgniteTxEx tx = cctx.tm().userTx();
+        IgniteInternalTx tx = cctx.tm().userTx();
 
         if (tx != null)
             throw new IllegalStateException("Failed to start new transaction " +
@@ -338,7 +338,7 @@ public class IgniteTransactionsImpl<K, V> implements IgniteTransactionsEx {
 
     /** {@inheritDoc} */
     @Nullable @Override public IgniteTx tx() {
-        IgniteTxEx tx = cctx.tm().userTx();
+        IgniteInternalTx tx = cctx.tm().userTx();
 
         return tx != null ? tx.proxy() : null;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index 48baf09..4de5441 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -51,7 +51,7 @@ import static org.apache.ignite.internal.processors.cache.GridCacheOperation.*;
  * Managed transaction adapter.
  */
 public abstract class IgniteTxAdapter<K, V> extends GridMetadataAwareAdapter
-    implements IgniteTxEx<K, V>, Externalizable {
+    implements IgniteInternalTx<K, V>, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -181,7 +181,7 @@ public abstract class IgniteTxAdapter<K, V> extends GridMetadataAwareAdapter
 
     /** */
     @GridToStringExclude
-    private AtomicReference<GridFutureAdapter<IgniteTxEx>> finFut = new AtomicReference<>();
+    private AtomicReference<GridFutureAdapter<IgniteInternalTx>> finFut = new AtomicReference<>();
 
     /** Topology version. */
     private AtomicLong topVer = new AtomicLong(-1);
@@ -940,11 +940,11 @@ public abstract class IgniteTxAdapter<K, V> extends GridMetadataAwareAdapter
 
     /** {@inheritDoc} */
     @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor")
-    @Override public IgniteInternalFuture<IgniteTxEx> finishFuture() {
-        GridFutureAdapter<IgniteTxEx> fut = finFut.get();
+    @Override public IgniteInternalFuture<IgniteInternalTx> finishFuture() {
+        GridFutureAdapter<IgniteInternalTx> fut = finFut.get();
 
         if (fut == null) {
-            fut = new GridFutureAdapter<IgniteTxEx>(cctx.kernalContext()) {
+            fut = new GridFutureAdapter<IgniteInternalTx>(cctx.kernalContext()) {
                 @Override public String toString() {
                     return S.toString(GridFutureAdapter.class, this, "tx", IgniteTxAdapter.this);
                 }
@@ -1066,7 +1066,7 @@ public abstract class IgniteTxAdapter<K, V> extends GridMetadataAwareAdapter
         }
 
         if (notify) {
-            GridFutureAdapter<IgniteTxEx> fut = finFut.get();
+            GridFutureAdapter<IgniteInternalTx> fut = finFut.get();
 
             if (fut != null)
                 fut.onDone(this);
@@ -1436,7 +1436,7 @@ public abstract class IgniteTxAdapter<K, V> extends GridMetadataAwareAdapter
     /**
      * Transaction shadow class to be used for deserialization.
      */
-    private static class TxShadow implements IgniteTxEx {
+    private static class TxShadow implements IgniteInternalTx {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -1869,7 +1869,7 @@ public abstract class IgniteTxAdapter<K, V> extends GridMetadataAwareAdapter
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteInternalFuture<IgniteTxEx> prepareAsync() {
+        @Override public IgniteInternalFuture<IgniteInternalTx> prepareAsync() {
             return null;
         }
 
@@ -1889,7 +1889,7 @@ public abstract class IgniteTxAdapter<K, V> extends GridMetadataAwareAdapter
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteInternalFuture<IgniteTxEx> finishFuture() {
+        @Override public IgniteInternalFuture<IgniteInternalTx> finishFuture() {
             return null;
         }
 
@@ -1914,12 +1914,12 @@ public abstract class IgniteTxAdapter<K, V> extends GridMetadataAwareAdapter
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteInternalFuture<IgniteTxEx> rollbackAsync() {
+        @Override public IgniteInternalFuture<IgniteInternalTx> rollbackAsync() {
             return null;
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteInternalFuture<IgniteTxEx> commitAsync() {
+        @Override public IgniteInternalFuture<IgniteInternalTx> commitAsync() {
             return null;
         }
 
@@ -2025,7 +2025,7 @@ public abstract class IgniteTxAdapter<K, V> extends GridMetadataAwareAdapter
 
         /** {@inheritDoc} */
         @Override public boolean equals(Object o) {
-            return this == o || o instanceof IgniteTxEx && xid.equals(((IgniteTxEx)o).xid());
+            return this == o || o instanceof IgniteInternalTx && xid.equals(((IgniteInternalTx)o).xid());
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
index 11fd32c..b129a79 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
@@ -53,7 +53,7 @@ public class IgniteTxEntry<K, V> implements GridPeerDeployAware, Externalizable,
 
     /** Owning transaction. */
     @GridToStringExclude
-    private IgniteTxEx<K, V> tx;
+    private IgniteInternalTx<K, V> tx;
 
     /** Cache key. */
     @GridToStringInclude
@@ -167,7 +167,7 @@ public class IgniteTxEntry<K, V> implements GridPeerDeployAware, Externalizable,
      * @param drVer Data center replication version.
      */
     public IgniteTxEntry(GridCacheContext<K, V> ctx,
-        IgniteTxEx<K, V> tx,
+        IgniteInternalTx<K, V> tx,
         GridCacheOperation op,
         V val,
         long ttl,
@@ -210,7 +210,7 @@ public class IgniteTxEntry<K, V> implements GridPeerDeployAware, Externalizable,
      * @param drVer Data center replication version.
      */
     public IgniteTxEntry(GridCacheContext<K, V> ctx,
-        IgniteTxEx<K, V> tx,
+        IgniteInternalTx<K, V> tx,
         GridCacheOperation op,
         V val,
         EntryProcessor<K, V, ?> entryProcessor,


[2/4] incubator-ignite git commit: # ignite-26 renamed IgniteTxEx - IgniteInternalTx

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEx.java
deleted file mode 100644
index f9aab18..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEx.java
+++ /dev/null
@@ -1,706 +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.transactions;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.internal.processors.cache.version.*;
-import org.apache.ignite.internal.transactions.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.transactions.*;
-import org.apache.ignite.internal.processors.timeout.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-/**
- * Transaction managed by cache ({@code 'Ex'} stands for external).
- */
-public interface IgniteTxEx<K, V> extends AutoCloseable, GridTimeoutObject {
-    /**
-     *
-     */
-    @SuppressWarnings("PublicInnerClass")
-    public enum FinalizationStatus {
-        /** Transaction was not finalized yet. */
-        NONE,
-
-        /** Transaction is being finalized by user. */
-        USER_FINISH,
-
-        /** Recovery request is received, user finish requests should be ignored. */
-        RECOVERY_WAIT,
-
-        /** Transaction is being finalized by recovery procedure. */
-        RECOVERY_FINISH
-    }
-
-    /**
-     * Gets unique identifier for this transaction.
-     *
-     * @return Transaction UID.
-     */
-    public IgniteUuid xid();
-
-    /**
-     * ID of the node on which this transaction started.
-     *
-     * @return Originating node ID.
-     */
-    public UUID nodeId();
-
-    /**
-     * ID of the thread in which this transaction started.
-     *
-     * @return Thread ID.
-     */
-    public long threadId();
-
-    /**
-     * Start time of this transaction.
-     *
-     * @return Start time of this transaction on this node.
-     */
-    public long startTime();
-
-    /**
-     * Cache transaction isolation level.
-     *
-     * @return Isolation level.
-     */
-    public IgniteTxIsolation isolation();
-
-    /**
-     * Cache transaction concurrency mode.
-     *
-     * @return Concurrency mode.
-     */
-    public IgniteTxConcurrency concurrency();
-
-    /**
-     * Flag indicating whether transaction was started automatically by the
-     * system or not. System will start transactions implicitly whenever
-     * any cache {@code put(..)} or {@code remove(..)} operation is invoked
-     * outside of transaction.
-     *
-     * @return {@code True} if transaction was started implicitly.
-     */
-    public boolean implicit();
-
-    /**
-     * Get invalidation flag for this transaction. If set to {@code true}, then
-     * remote values will be {@code invalidated} (set to {@code null}) instead
-     * of updated.
-     * <p>
-     * Invalidation messages don't carry new values, so they are a lot lighter
-     * than update messages. However, when a value is accessed on a node after
-     * it's been invalidated, it must be loaded from persistent store.
-     *
-     * @return Invalidation flag.
-     */
-    public boolean isInvalidate();
-
-    /**
-     * Gets current transaction state value.
-     *
-     * @return Current transaction state.
-     */
-    public IgniteTxState state();
-
-    /**
-     * Gets timeout value in milliseconds for this transaction. If transaction times
-     * out prior to it's completion, {@link org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException} will be thrown.
-     *
-     * @return Transaction timeout value.
-     */
-    public long timeout();
-
-    /**
-     * Sets transaction timeout value. This value can be set only before a first operation
-     * on transaction has been performed.
-     *
-     * @param timeout Transaction timeout value.
-     * @return Previous timeout.
-     */
-    public long timeout(long timeout);
-
-    /**
-     * Modify the transaction associated with the current thread such that the
-     * only possible outcome of the transaction is to roll back the
-     * transaction.
-     *
-     * @return {@code True} if rollback-only flag was set as a result of this operation,
-     *      {@code false} if it was already set prior to this call or could not be set
-     *      because transaction is already finishing up committing or rolling back.
-     */
-    public boolean setRollbackOnly();
-
-    /**
-     * If transaction was marked as rollback-only.
-     *
-     * @return {@code True} if transaction can only be rolled back.
-     */
-    public boolean isRollbackOnly();
-
-    /**
-     * Commits this transaction by initiating {@code two-phase-commit} process.
-     *
-     * @throws IgniteCheckedException If commit failed.
-     */
-    @IgniteAsyncSupported
-    public void commit() throws IgniteCheckedException;
-
-    /**
-     * Ends the transaction. Transaction will be rolled back if it has not been committed.
-     *
-     * @throws IgniteCheckedException If transaction could not be gracefully ended.
-     */
-    @Override public void close() throws IgniteCheckedException;
-
-    /**
-     * Rolls back this transaction.
-     *
-     * @throws IgniteCheckedException If rollback failed.
-     */
-    @IgniteAsyncSupported
-    public void rollback() throws IgniteCheckedException;
-
-    /**
-     * Removes metadata by name.
-     *
-     * @param name Name of the metadata to remove.
-     * @param <T> Type of the value.
-     * @return Value of removed metadata or {@code null}.
-     */
-    @Nullable public <T> T removeMeta(String name);
-
-    /**
-     * Gets metadata by name.
-     *
-     * @param name Metadata name.
-     * @param <T> Type of the value.
-     * @return Metadata value or {@code null}.
-     */
-    @Nullable public <T> T meta(String name);
-
-    /**
-     * Adds a new metadata.
-     *
-     * @param name Metadata name.
-     * @param val Metadata value.
-     * @param <T> Type of the value.
-     * @return Metadata previously associated with given name, or
-     *      {@code null} if there was none.
-     */
-    @Nullable public <T> T addMeta(String name, T val);
-
-    /**
-     * @return Size of the transaction.
-     */
-    public int size();
-
-    /**
-     * @return {@code True} if transaction is allowed to use store.
-     */
-    public boolean storeEnabled();
-
-    /**
-     * @return {@code True} if transaction is allowed to use store and transactions spans one or more caches with
-     *      store enabled.
-     */
-    public boolean storeUsed();
-
-    /**
-     * Checks if this is system cache transaction. System transactions are isolated from user transactions
-     * because some of the public API methods may be invoked inside user transactions and internally start
-     * system cache transactions.
-     *
-     * @return {@code True} if transaction is started for system cache.
-     */
-    public boolean system();
-
-    /**
-     * @return Last recorded topology version.
-     */
-    public long topologyVersion();
-
-    /**
-     * @return Flag indicating whether transaction is implicit with only one key.
-     */
-    public boolean implicitSingle();
-
-    /**
-     * @return Collection of cache IDs involved in this transaction.
-     */
-    public Collection<Integer> activeCacheIds();
-
-    /**
-     * Attempts to set topology version and returns the current value.
-     * If topology version was previously set, then it's value will
-     * be returned (but not updated).
-     *
-     * @param topVer Topology version.
-     * @return Recorded topology version.
-     */
-    public long topologyVersion(long topVer);
-
-    /**
-     * @return {@code True} if transaction is empty.
-     */
-    public boolean empty();
-
-    /**
-     * @return {@code True} if transaction group-locked.
-     */
-    public boolean groupLock();
-
-    /**
-     * @return Group lock key if {@link #groupLock()} is {@code true}.
-     */
-    @Nullable public IgniteTxKey groupLockKey();
-
-    /**
-     * @return {@code True} if preparing flag was set with this call.
-     */
-    public boolean markPreparing();
-
-    /**
-     * @param status Finalization status to set.
-     * @return {@code True} if could mark was set.
-     */
-    public boolean markFinalizing(FinalizationStatus status);
-
-    /**
-     * @param cacheCtx Cache context.
-     * @param part Invalid partition.
-     */
-    public void addInvalidPartition(GridCacheContext<K, V> cacheCtx, int part);
-
-    /**
-     * @return Invalid partitions.
-     */
-    public Set<Integer> invalidPartitions();
-
-    /**
-     * Gets owned version for near remote transaction.
-     *
-     * @param key Key to get version for.
-     * @return Owned version, if any.
-     */
-    @Nullable public GridCacheVersion ownedVersion(IgniteTxKey<K> key);
-
-    /**
-     * Gets ID of additional node involved. For example, in DHT case, other node is
-     * near node ID.
-     *
-     * @return Parent node IDs.
-     */
-    @Nullable public UUID otherNodeId();
-
-    /**
-     * @return Event node ID.
-     */
-    public UUID eventNodeId();
-
-    /**
-     * Gets node ID which directly started this transaction. In case of DHT local transaction it will be
-     * near node ID, in case of DHT remote transaction it will be primary node ID, in case of replicated remote
-     * transaction it will be starter node ID.
-     *
-     * @return Originating node ID.
-     */
-    public UUID originatingNodeId();
-
-    /**
-     * @return Master node IDs.
-     */
-    public Collection<UUID> masterNodeIds();
-
-    /**
-     * @return Near transaction ID.
-     */
-    @Nullable public GridCacheVersion nearXidVersion();
-
-    /**
-     * @return Transaction nodes mapping (primary node -> related backup nodes).
-     */
-    @Nullable public Map<UUID, Collection<UUID>> transactionNodes();
-
-    /**
-     * @param entry Entry to check.
-     * @return {@code True} if lock is owned.
-     * @throws GridCacheEntryRemovedException If entry has been removed.
-     */
-    public boolean ownsLock(GridCacheEntryEx<K, V> entry) throws GridCacheEntryRemovedException;
-
-    /**
-     * @param entry Entry to check.
-     * @return {@code True} if lock is owned.
-     */
-    public boolean ownsLockUnsafe(GridCacheEntryEx<K, V> entry);
-
-    /**
-     * For Partitioned caches, this flag is {@code false} for remote DHT and remote NEAR
-     * transactions because serializability of transaction is enforced on primary node. All
-     * other transaction types must enforce it.
-     *
-     * @return Enforce serializable flag.
-     */
-    public boolean enforceSerializable();
-
-    /**
-     * @return {@code True} if near transaction.
-     */
-    public boolean near();
-
-    /**
-     * @return {@code True} if DHT transaction.
-     */
-    public boolean dht();
-
-    /**
-     * @return {@code True} if dht colocated transaction.
-     */
-    public boolean colocated();
-
-    /**
-     * @return {@code True} if transaction is local, {@code false} if it's remote.
-     */
-    public boolean local();
-
-    /**
-     * @return {@code True} if transaction is replicated.
-     */
-    public boolean replicated();
-
-    /**
-     * @return Subject ID initiated this transaction.
-     */
-    public UUID subjectId();
-
-    /**
-     * Task name hash in case if transaction was initiated within task execution.
-     *
-     * @return Task name hash.
-     */
-    public int taskNameHash();
-
-    /**
-     * @return {@code True} if transaction is user transaction, which means:
-     * <ul>
-     *     <li>Explicit</li>
-     *     <li>Local</li>
-     *     <li>Not DHT</li>
-     * </ul>
-     */
-    public boolean user();
-
-    /**
-     * @return {@code True} if transaction is configured with synchronous commit flag.
-     */
-    public boolean syncCommit();
-
-    /**
-     * @return {@code True} if transaction is configured with synchronous rollback flag.
-     */
-    public boolean syncRollback();
-
-    /**
-     * @param key Key to check.
-     * @return {@code True} if key is present.
-     */
-    public boolean hasWriteKey(IgniteTxKey<K> key);
-
-    /**
-     * @return Read set.
-     */
-    public Set<IgniteTxKey<K>> readSet();
-
-    /**
-     * @return Write set.
-     */
-    public Set<IgniteTxKey<K>> writeSet();
-
-    /**
-     * @return All transaction entries.
-     */
-    public Collection<IgniteTxEntry<K, V>> allEntries();
-
-    /**
-     * @return Write entries.
-     */
-    public Collection<IgniteTxEntry<K, V>> writeEntries();
-
-    /**
-     * @return Read entries.
-     */
-    public Collection<IgniteTxEntry<K, V>> readEntries();
-
-    /**
-     * @return Transaction write map.
-     */
-    public Map<IgniteTxKey<K>, IgniteTxEntry<K, V>> writeMap();
-
-    /**
-     * @return Transaction read map.
-     */
-    public Map<IgniteTxKey<K>, IgniteTxEntry<K, V>> readMap();
-
-    /**
-     * Gets pessimistic recovery writes, i.e. values that have never been sent to remote nodes with lock requests.
-     *
-     * @return Collection of recovery writes.
-     */
-    public Collection<IgniteTxEntry<K, V>> recoveryWrites();
-
-    /**
-     * Gets a list of entries that needs to be locked on the next step of prepare stage of
-     * optimistic transaction.
-     *
-     * @return List of tx entries for optimistic locking.
-     */
-    public Collection<IgniteTxEntry<K, V>> optimisticLockEntries();
-
-    /**
-     * Seals transaction for updates.
-     */
-    public void seal();
-
-    /**
-     * @param key Key for the entry.
-     * @return Entry for the key (either from write set or read set).
-     */
-    @Nullable public IgniteTxEntry<K, V> entry(IgniteTxKey<K> key);
-
-    /**
-     * @param ctx Cache context.
-     * @param failFast Fail-fast flag.
-     * @param key Key to look up.
-     * @param filter Filter to check.
-     * @return Current value for the key within transaction.
-     * @throws GridCacheFilterFailedException If filter failed and failFast is {@code true}.
-     */
-     @Nullable public GridTuple<V> peek(
-         GridCacheContext<K, V> ctx,
-         boolean failFast,
-         K key,
-         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) throws GridCacheFilterFailedException;
-
-    /**
-     * @return Start version.
-     */
-    public GridCacheVersion startVersion();
-
-    /**
-     * @return Transaction version.
-     */
-    public GridCacheVersion xidVersion();
-
-    /**
-     * @return Version created at commit time.
-     */
-    public GridCacheVersion commitVersion();
-
-    /**
-     * @param commitVer Commit version.
-     * @return {@code True} if version was set.
-     */
-    public boolean commitVersion(GridCacheVersion commitVer);
-
-    /**
-     * @return End version (a.k.a. <tt>'tnc'</tt> or <tt>'transaction number counter'</tt>)
-     *      assigned to this transaction at the end of write phase.
-     */
-    public GridCacheVersion endVersion();
-
-    /**
-     * Prepare state.
-     *
-     * @throws IgniteCheckedException If failed.
-     */
-    public void prepare() throws IgniteCheckedException;
-
-    /**
-     * Prepare stage.
-     *
-     * @return Future for prepare step.
-     */
-    public IgniteInternalFuture<IgniteTxEx<K, V>> prepareAsync();
-
-    /**
-     * @param endVer End version (a.k.a. <tt>'tnc'</tt> or <tt>'transaction number counter'</tt>)
-     *      assigned to this transaction at the end of write phase.
-     */
-    public void endVersion(GridCacheVersion endVer);
-
-    /**
-     * @return Transaction write version. For all transactions except DHT transactions, will be equal to
-     *      {@link #xidVersion()}.
-     */
-    public GridCacheVersion writeVersion();
-
-    /**
-     * Sets write version.
-     *
-     * @param ver Write version.
-     */
-    public void writeVersion(GridCacheVersion ver);
-
-    /**
-     * @return Future for transaction completion.
-     */
-    public IgniteInternalFuture<IgniteTxEx> finishFuture();
-
-    /**
-     * @param state Transaction state.
-     * @return {@code True} if transition was valid, {@code false} otherwise.
-     */
-    public boolean state(IgniteTxState state);
-
-    /**
-     * @param invalidate Invalidate flag.
-     */
-    public void invalidate(boolean invalidate);
-
-    /**
-     * @param sysInvalidate System invalidate flag.
-     */
-    public void systemInvalidate(boolean sysInvalidate);
-
-    /**
-     * @return System invalidate flag.
-     */
-    public boolean isSystemInvalidate();
-
-    /**
-     * Asynchronously rollback this transaction.
-     *
-     * @return Rollback future.
-     */
-    public IgniteInternalFuture<IgniteTxEx> rollbackAsync();
-
-    /**
-     * Asynchronously commits this transaction by initiating {@code two-phase-commit} process.
-     *
-     * @return Future for commit operation.
-     */
-    public IgniteInternalFuture<IgniteTxEx> commitAsync();
-
-    /**
-     * Callback invoked whenever there is a lock that has been acquired
-     * by this transaction for any of the participating entries.
-     *
-     * @param entry Cache entry.
-     * @param owner Lock candidate that won ownership of the lock.
-     * @return {@code True} if transaction cared about notification.
-     */
-    public boolean onOwnerChanged(GridCacheEntryEx<K, V> entry, GridCacheMvccCandidate<K> owner);
-
-    /**
-     * @return {@code True} if transaction timed out.
-     */
-    public boolean timedOut();
-
-    /**
-     * @return {@code True} if transaction had completed successfully or unsuccessfully.
-     */
-    public boolean done();
-
-    /**
-     * @return {@code True} for OPTIMISTIC transactions.
-     */
-    public boolean optimistic();
-
-    /**
-     * @return {@code True} for PESSIMISTIC transactions.
-     */
-    public boolean pessimistic();
-
-    /**
-     * @return {@code True} if read-committed.
-     */
-    public boolean readCommitted();
-
-    /**
-     * @return {@code True} if repeatable-read.
-     */
-    public boolean repeatableRead();
-
-    /**
-     * @return {@code True} if serializable.
-     */
-    public boolean serializable();
-
-    /**
-     * Checks whether given key has been removed within transaction.
-     *
-     * @param key Key to check.
-     * @return {@code True} if key has been removed.
-     */
-    public boolean removed(IgniteTxKey<K> key);
-
-    /**
-     * Gets allowed remaining time for this transaction.
-     *
-     * @return Remaining time.
-     * @throws org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException If transaction timed out.
-     */
-    public long remainingTime() throws IgniteTxTimeoutCheckedException;
-
-    /**
-     * @return Alternate transaction versions.
-     */
-    public Collection<GridCacheVersion> alternateVersions();
-
-    /**
-     * @return {@code True} if transaction needs completed versions for processing.
-     */
-    public boolean needsCompletedVersions();
-
-    /**
-     * @param base Base for committed versions.
-     * @param committed Committed transactions relative to base.
-     * @param rolledback Rolled back transactions relative to base.
-     */
-    public void completedVersions(GridCacheVersion base, Collection<GridCacheVersion> committed,
-        Collection<GridCacheVersion> rolledback);
-
-    /**
-     * @return {@code True} if transaction has at least one internal entry.
-     */
-    public boolean internal();
-
-    /**
-     * @return {@code True} if transaction is a one-phase-commit transaction.
-     */
-    public boolean onePhaseCommit();
-
-    /**
-     * @return {@code True} if transaction has transform entries. This flag will be only set for local
-     *      transactions.
-     */
-    public boolean hasTransforms();
-
-    /**
-     * @return Public API proxy.
-     */
-    public IgniteTxProxy proxy();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/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 4cceb7f..832438d 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
@@ -40,7 +40,7 @@ import static org.apache.ignite.transactions.IgniteTxConcurrency.*;
 import static org.apache.ignite.transactions.IgniteTxIsolation.*;
 import static org.apache.ignite.transactions.IgniteTxState.*;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.*;
-import static org.apache.ignite.internal.processors.cache.transactions.IgniteTxEx.FinalizationStatus.*;
+import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.*;
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.*;
 
 /**
@@ -53,7 +53,7 @@ public class IgniteTxHandler<K, V> {
     /** Shared cache context. */
     private GridCacheSharedContext<K, V> ctx;
 
-    public IgniteInternalFuture<IgniteTxEx<K, V>> processNearTxPrepareRequest(final UUID nearNodeId,
+    public IgniteInternalFuture<IgniteInternalTx<K, V>> processNearTxPrepareRequest(final UUID nearNodeId,
         final GridNearTxPrepareRequest<K, V> req) {
         return prepareTx(nearNodeId, null, req);
     }
@@ -149,7 +149,7 @@ public class IgniteTxHandler<K, V> {
      * @param req Near prepare request.
      * @return Future for transaction.
      */
-    public IgniteInternalFuture<IgniteTxEx<K, V>> prepareTx(final UUID nearNodeId, @Nullable GridNearTxLocal<K, V> locTx,
+    public IgniteInternalFuture<IgniteInternalTx<K, V>> prepareTx(final UUID nearNodeId, @Nullable GridNearTxLocal<K, V> locTx,
         final GridNearTxPrepareRequest<K, V> req) {
         assert nearNodeId != null;
         assert req != null;
@@ -175,7 +175,7 @@ public class IgniteTxHandler<K, V> {
      * @param req Near prepare request.
      * @return Prepare future.
      */
-    private IgniteInternalFuture<IgniteTxEx<K, V>> prepareColocatedTx(final GridNearTxLocal<K, V> locTx,
+    private IgniteInternalFuture<IgniteInternalTx<K, V>> prepareColocatedTx(final GridNearTxLocal<K, V> locTx,
         final GridNearTxPrepareRequest<K, V> req) {
 
         IgniteInternalFuture<Object> fut = new GridFinishedFutureEx<>(); // TODO force preload keys.
@@ -183,12 +183,12 @@ public class IgniteTxHandler<K, V> {
         return new GridEmbeddedFuture<>(
             ctx.kernalContext(),
             fut,
-            new C2<Object, Exception, IgniteInternalFuture<IgniteTxEx<K, V>>>() {
-                @Override public IgniteInternalFuture<IgniteTxEx<K, V>> apply(Object o, Exception ex) {
+            new C2<Object, Exception, IgniteInternalFuture<IgniteInternalTx<K, V>>>() {
+                @Override public IgniteInternalFuture<IgniteInternalTx<K, V>> apply(Object o, Exception ex) {
                     if (ex != null)
                         throw new GridClosureException(ex);
 
-                    IgniteInternalFuture<IgniteTxEx<K, V>> fut = locTx.prepareAsyncLocal(req.reads(), req.writes(),
+                    IgniteInternalFuture<IgniteInternalTx<K, V>> fut = locTx.prepareAsyncLocal(req.reads(), req.writes(),
                         req.transactionNodes(), req.last(), req.lastBackups());
 
                     if (locTx.isRollbackOnly())
@@ -197,8 +197,8 @@ public class IgniteTxHandler<K, V> {
                     return fut;
                 }
             },
-            new C2<IgniteTxEx<K, V>, Exception, IgniteTxEx<K, V>>() {
-                @Nullable @Override public IgniteTxEx<K, V> apply(IgniteTxEx<K, V> tx, Exception e) {
+            new C2<IgniteInternalTx<K, V>, Exception, IgniteInternalTx<K, V>>() {
+                @Nullable @Override public IgniteInternalTx<K, V> apply(IgniteInternalTx<K, V> tx, Exception e) {
                     if (e != null) {
                         // tx can be null of exception occurred.
                         if (tx != null)
@@ -221,7 +221,7 @@ public class IgniteTxHandler<K, V> {
      * @param req Near prepare request.
      * @return Prepare future.
      */
-    private IgniteInternalFuture<IgniteTxEx<K, V>> prepareNearTx(final UUID nearNodeId,
+    private IgniteInternalFuture<IgniteInternalTx<K, V>> prepareNearTx(final UUID nearNodeId,
         final GridNearTxPrepareRequest<K, V> req) {
         ClusterNode nearNode = ctx.node(nearNodeId);
 
@@ -285,7 +285,7 @@ public class IgniteTxHandler<K, V> {
         }
 
         if (tx != null) {
-            IgniteInternalFuture<IgniteTxEx<K, V>> fut = tx.prepareAsync(req.reads(), req.writes(),
+            IgniteInternalFuture<IgniteInternalTx<K, V>> fut = tx.prepareAsync(req.reads(), req.writes(),
                 req.dhtVersions(), req.messageId(), req.miniId(), req.transactionNodes(), req.last(),
                 req.lastBackups());
 
@@ -300,8 +300,8 @@ public class IgniteTxHandler<K, V> {
 
             final GridDhtTxLocal<K, V> tx0 = tx;
 
-            fut.listenAsync(new CI1<IgniteInternalFuture<IgniteTxEx<K, V>>>() {
-                @Override public void apply(IgniteInternalFuture<IgniteTxEx<K, V>> txFut) {
+            fut.listenAsync(new CI1<IgniteInternalFuture<IgniteInternalTx<K, V>>>() {
+                @Override public void apply(IgniteInternalFuture<IgniteInternalTx<K, V>> txFut) {
                     try {
                         txFut.get();
                     }
@@ -317,7 +317,7 @@ public class IgniteTxHandler<K, V> {
             return fut;
         }
         else
-            return new GridFinishedFuture<>(ctx.kernalContext(), (IgniteTxEx<K, V>)null);
+            return new GridFinishedFuture<>(ctx.kernalContext(), (IgniteInternalTx<K, V>)null);
     }
 
     /**
@@ -326,7 +326,7 @@ public class IgniteTxHandler<K, V> {
      */
     private void processNearTxPrepareResponse(UUID nodeId, GridNearTxPrepareResponse<K, V> res) {
         GridNearTxPrepareFuture<K, V> fut = (GridNearTxPrepareFuture<K, V>)ctx.mvcc()
-            .<IgniteTxEx<K, V>>future(res.version(), res.futureId());
+            .<IgniteInternalTx<K, V>>future(res.version(), res.futureId());
 
         if (fut == null) {
             if (log.isDebugEnabled())
@@ -345,7 +345,7 @@ public class IgniteTxHandler<K, V> {
     private void processNearTxFinishResponse(UUID nodeId, GridNearTxFinishResponse<K, V> res) {
         ctx.tm().onFinishedRemote(nodeId, res.threadId());
 
-        GridNearTxFinishFuture<K, V> fut = (GridNearTxFinishFuture<K, V>)ctx.mvcc().<IgniteTxEx>future(
+        GridNearTxFinishFuture<K, V> fut = (GridNearTxFinishFuture<K, V>)ctx.mvcc().<IgniteInternalTx>future(
             res.xid(), res.futureId());
 
         if (fut == null) {
@@ -364,7 +364,7 @@ public class IgniteTxHandler<K, V> {
      */
     private void processDhtTxPrepareResponse(UUID nodeId, GridDhtTxPrepareResponse<K, V> res) {
         GridDhtTxPrepareFuture<K, V> fut = (GridDhtTxPrepareFuture<K, V>)ctx.mvcc().
-            <IgniteTxEx<K, V>>future(res.version(), res.futureId());
+            <IgniteInternalTx<K, V>>future(res.version(), res.futureId());
 
         if (fut == null) {
             if (log.isDebugEnabled())
@@ -384,7 +384,7 @@ public class IgniteTxHandler<K, V> {
         assert nodeId != null;
         assert res != null;
 
-        GridDhtTxFinishFuture<K, V> fut = (GridDhtTxFinishFuture<K, V>)ctx.mvcc().<IgniteTxEx>future(res.xid(),
+        GridDhtTxFinishFuture<K, V> fut = (GridDhtTxFinishFuture<K, V>)ctx.mvcc().<IgniteInternalTx>future(res.xid(),
             res.futureId());
 
         if (fut == null) {
@@ -402,7 +402,7 @@ public class IgniteTxHandler<K, V> {
      * @param req Request.
      * @return Future.
      */
-    @Nullable public IgniteInternalFuture<IgniteTxEx> processNearTxFinishRequest(UUID nodeId, GridNearTxFinishRequest<K, V> req) {
+    @Nullable public IgniteInternalFuture<IgniteInternalTx> processNearTxFinishRequest(UUID nodeId, GridNearTxFinishRequest<K, V> req) {
         return finish(nodeId, null, req);
     }
 
@@ -411,24 +411,24 @@ public class IgniteTxHandler<K, V> {
      * @param req Request.
      * @return Future.
      */
-    @Nullable public IgniteInternalFuture<IgniteTxEx> finish(UUID nodeId, @Nullable GridNearTxLocal<K, V> locTx,
+    @Nullable public IgniteInternalFuture<IgniteInternalTx> finish(UUID nodeId, @Nullable GridNearTxLocal<K, V> locTx,
         GridNearTxFinishRequest<K, V> req) {
         assert nodeId != null;
         assert req != null;
 
         // Transaction on local cache only.
         if (locTx != null && !locTx.nearLocallyMapped() && !locTx.colocatedLocallyMapped())
-            return new GridFinishedFutureEx<IgniteTxEx>(locTx);
+            return new GridFinishedFutureEx<IgniteInternalTx>(locTx);
 
         if (log.isDebugEnabled())
             log.debug("Processing near tx finish request [nodeId=" + nodeId + ", req=" + req + "]");
 
-        IgniteInternalFuture<IgniteTxEx> colocatedFinishFut = null;
+        IgniteInternalFuture<IgniteInternalTx> colocatedFinishFut = null;
 
         if (locTx != null && locTx.colocatedLocallyMapped())
             colocatedFinishFut = finishColocatedLocal(req.commit(), locTx);
 
-        IgniteInternalFuture<IgniteTxEx> nearFinishFut = null;
+        IgniteInternalFuture<IgniteInternalTx> nearFinishFut = null;
 
         if (locTx == null || locTx.nearLocallyMapped()) {
             if (locTx != null)
@@ -438,7 +438,7 @@ public class IgniteTxHandler<K, V> {
         }
 
         if (colocatedFinishFut != null && nearFinishFut != null) {
-            GridCompoundFuture<IgniteTxEx, IgniteTxEx> res = new GridCompoundFuture<>(ctx.kernalContext());
+            GridCompoundFuture<IgniteInternalTx, IgniteInternalTx> res = new GridCompoundFuture<>(ctx.kernalContext());
 
             res.add(colocatedFinishFut);
             res.add(nearFinishFut);
@@ -460,7 +460,7 @@ public class IgniteTxHandler<K, V> {
      * @param req Finish request.
      * @return Finish future.
      */
-    private IgniteInternalFuture<IgniteTxEx> finishDhtLocal(UUID nodeId, @Nullable GridNearTxLocal<K, V> locTx,
+    private IgniteInternalFuture<IgniteInternalTx> finishDhtLocal(UUID nodeId, @Nullable GridNearTxLocal<K, V> locTx,
         GridNearTxFinishRequest<K, V> req) {
         GridCacheVersion dhtVer = ctx.tm().mappedVersion(req.version());
 
@@ -564,7 +564,7 @@ public class IgniteTxHandler<K, V> {
                 if (tx.pessimistic())
                     tx.prepare();
 
-                IgniteInternalFuture<IgniteTxEx> commitFut = tx.commitAsync();
+                IgniteInternalFuture<IgniteInternalTx> commitFut = tx.commitAsync();
 
                 // Only for error logging.
                 commitFut.listenAsync(CU.errorLogger(log));
@@ -580,7 +580,7 @@ public class IgniteTxHandler<K, V> {
                 tx.nearFinishFutureId(req.futureId());
                 tx.nearFinishMiniId(req.miniId());
 
-                IgniteInternalFuture<IgniteTxEx> rollbackFut = tx.rollbackAsync();
+                IgniteInternalFuture<IgniteInternalTx> rollbackFut = tx.rollbackAsync();
 
                 // Only for error logging.
                 rollbackFut.listenAsync(CU.errorLogger(log));
@@ -592,7 +592,7 @@ public class IgniteTxHandler<K, V> {
             U.error(log, "Failed completing transaction [commit=" + req.commit() + ", tx=" + tx + ']', e);
 
             if (tx != null) {
-                IgniteInternalFuture<IgniteTxEx> rollbackFut = tx.rollbackAsync();
+                IgniteInternalFuture<IgniteInternalTx> rollbackFut = tx.rollbackAsync();
 
                 // Only for error logging.
                 rollbackFut.listenAsync(CU.errorLogger(log));
@@ -609,7 +609,7 @@ public class IgniteTxHandler<K, V> {
      * @param tx Transaction to commit.
      * @return Future.
      */
-    public IgniteInternalFuture<IgniteTxEx> finishColocatedLocal(boolean commit, GridNearTxLocal<K, V> tx) {
+    public IgniteInternalFuture<IgniteInternalTx> finishColocatedLocal(boolean commit, GridNearTxLocal<K, V> tx) {
         try {
             if (commit) {
                 if (!tx.markFinalizing(USER_FINISH)) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/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 d110543..2a6fd9a 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
@@ -3421,8 +3421,8 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                 setRollbackOnly();
 
                 if (commit && commitAfterLock())
-                    return rollbackAsync().chain(new C1<IgniteInternalFuture<IgniteTxEx>, T>() {
-                        @Override public T apply(IgniteInternalFuture<IgniteTxEx> f) {
+                    return rollbackAsync().chain(new C1<IgniteInternalFuture<IgniteInternalTx>, T>() {
+                        @Override public T apply(IgniteInternalFuture<IgniteInternalTx> f) {
                             throw new GridClosureException(e);
                         }
                     });
@@ -3438,8 +3438,8 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                     ", tx=" + this + ']'));
 
                 if (commit && commitAfterLock())
-                    return rollbackAsync().chain(new C1<IgniteInternalFuture<IgniteTxEx>, T>() {
-                        @Override public T apply(IgniteInternalFuture<IgniteTxEx> f) {
+                    return rollbackAsync().chain(new C1<IgniteInternalFuture<IgniteInternalTx>, T>() {
+                        @Override public T apply(IgniteInternalFuture<IgniteInternalTx> f) {
                             throw ex;
                         }
                     });
@@ -3456,8 +3456,8 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                 if (commit && commitAfterLock()) {
                     rollback = false;
 
-                    return commitAsync().chain(new CX1<IgniteInternalFuture<IgniteTxEx>, T>() {
-                        @Override public T applyx(IgniteInternalFuture<IgniteTxEx> f) throws IgniteCheckedException {
+                    return commitAsync().chain(new CX1<IgniteInternalFuture<IgniteInternalTx>, T>() {
+                        @Override public T applyx(IgniteInternalFuture<IgniteInternalTx> f) throws IgniteCheckedException {
                             f.get();
 
                             return r;
@@ -3471,8 +3471,8 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
             }
             catch (final IgniteCheckedException ex) {
                 if (commit && commitAfterLock())
-                    return rollbackAsync().chain(new C1<IgniteInternalFuture<IgniteTxEx>, T>() {
-                        @Override public T apply(IgniteInternalFuture<IgniteTxEx> f) {
+                    return rollbackAsync().chain(new C1<IgniteInternalFuture<IgniteInternalTx>, T>() {
+                        @Override public T apply(IgniteInternalFuture<IgniteInternalTx> f) {
                             throw new GridClosureException(ex);
                         }
                     });

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java
index f7ace8b..597d21e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java
@@ -32,7 +32,7 @@ import java.util.*;
 /**
  * Local transaction API.
  */
-public interface IgniteTxLocalEx<K, V> extends IgniteTxEx<K, V> {
+public interface IgniteTxLocalEx<K, V> extends IgniteInternalTx<K, V> {
     /**
      * @return Minimum version involved in transaction.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/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 0e7c0ee..fbe86e2 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
@@ -47,7 +47,7 @@ import static org.apache.ignite.IgniteSystemProperties.*;
 import static org.apache.ignite.events.IgniteEventType.*;
 import static org.apache.ignite.transactions.IgniteTxConcurrency.*;
 import static org.apache.ignite.transactions.IgniteTxState.*;
-import static org.apache.ignite.internal.processors.cache.transactions.IgniteTxEx.FinalizationStatus.*;
+import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.*;
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.*;
 import static org.apache.ignite.internal.util.GridConcurrentFactory.*;
 
@@ -65,25 +65,25 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
     private static final int TX_SALVAGE_TIMEOUT = Integer.getInteger(IGNITE_TX_SALVAGE_TIMEOUT, 100);
 
     /** Committing transactions. */
-    private final ThreadLocal<IgniteTxEx> threadCtx = new GridThreadLocalEx<>();
+    private final ThreadLocal<IgniteInternalTx> threadCtx = new GridThreadLocalEx<>();
 
     /** Per-thread transaction map. */
-    private final ConcurrentMap<Long, IgniteTxEx<K, V>> threadMap = newMap();
+    private final ConcurrentMap<Long, IgniteInternalTx<K, V>> threadMap = newMap();
 
     /** Per-ID map. */
-    private final ConcurrentMap<GridCacheVersion, IgniteTxEx<K, V>> idMap = newMap();
+    private final ConcurrentMap<GridCacheVersion, IgniteInternalTx<K, V>> idMap = newMap();
 
     /** Per-ID map for near transactions. */
-    private final ConcurrentMap<GridCacheVersion, IgniteTxEx<K, V>> nearIdMap = newMap();
+    private final ConcurrentMap<GridCacheVersion, IgniteInternalTx<K, V>> nearIdMap = newMap();
 
     /** TX handler. */
     private IgniteTxHandler<K, V> txHandler;
 
     /** All transactions. */
-    private final Queue<IgniteTxEx<K, V>> committedQ = new ConcurrentLinkedDeque8<>();
+    private final Queue<IgniteInternalTx<K, V>> committedQ = new ConcurrentLinkedDeque8<>();
 
     /** Preparing transactions. */
-    private final Queue<IgniteTxEx<K, V>> prepareQ = new ConcurrentLinkedDeque8<>();
+    private final Queue<IgniteInternalTx<K, V>> prepareQ = new ConcurrentLinkedDeque8<>();
 
     /** Minimum start version. */
     private final ConcurrentNavigableMap<GridCacheVersion, AtomicInt> startVerCnts =
@@ -138,7 +138,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
             },
             EVT_NODE_FAILED, EVT_NODE_LEFT);
 
-        for (IgniteTxEx<K, V> tx : idMap.values()) {
+        for (IgniteInternalTx<K, V> tx : idMap.values()) {
             if ((!tx.local() || tx.dht()) && !cctx.discovery().aliveAll(tx.masterNodeIds())) {
                 if (log.isDebugEnabled())
                     log.debug("Remaining transaction from left node: " + tx);
@@ -170,7 +170,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @param tx Transaction.
      * @return {@code True} if transaction was salvaged by this call.
      */
-    public boolean salvageTx(IgniteTxEx<K, V> tx) {
+    public boolean salvageTx(IgniteInternalTx<K, V> tx) {
         return salvageTx(tx, false, USER_FINISH);
     }
 
@@ -182,7 +182,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @param status Finalization status.
      * @return {@code True} if transaction was salvaged by this call.
      */
-    private boolean salvageTx(IgniteTxEx<K, V> tx, boolean warn, IgniteTxEx.FinalizationStatus status) {
+    private boolean salvageTx(IgniteInternalTx<K, V> tx, boolean warn, IgniteInternalTx.FinalizationStatus status) {
         assert tx != null;
 
         IgniteTxState state = tx.state();
@@ -258,7 +258,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * USE ONLY FOR MEMORY PROFILING DURING TESTS.
      */
     @Override public void printMemoryStats() {
-        IgniteTxEx<K, V> firstTx = committedQ.peek();
+        IgniteInternalTx<K, V> firstTx = committedQ.peek();
 
         int committedSize = committedQ.size();
 
@@ -270,9 +270,9 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
         if (committedSize > 3000) {
             minStartVer = new GridCacheVersion(Integer.MAX_VALUE, Long.MAX_VALUE, Long.MAX_VALUE, Integer.MAX_VALUE, 0);
 
-            IgniteTxEx<K, V> stuck = null;
+            IgniteInternalTx<K, V> stuck = null;
 
-            for (IgniteTxEx<K, V> tx : txs())
+            for (IgniteInternalTx<K, V> tx : txs())
                 if (tx.startVersion().isLess(minStartVer)) {
                     minStartVer = tx.startVersion();
                     dur = U.currentTimeMillis() - tx.startTime();
@@ -355,7 +355,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @return {@code True} if transaction has been committed or rolled back,
      *      {@code false} otherwise.
      */
-    public boolean isCompleted(IgniteTxEx<K, V> tx) {
+    public boolean isCompleted(IgniteInternalTx<K, V> tx) {
         return committedVers.contains(tx.xidVersion()) || rolledbackVers.contains(tx.xidVersion());
     }
 
@@ -409,8 +409,8 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @param tx Created transaction.
      * @return Started transaction.
      */
-    @Nullable public <T extends IgniteTxEx<K, V>> T onCreated(T tx) {
-        ConcurrentMap<GridCacheVersion, IgniteTxEx<K, V>> txIdMap = transactionMap(tx);
+    @Nullable public <T extends IgniteInternalTx<K, V>> T onCreated(T tx) {
+        ConcurrentMap<GridCacheVersion, IgniteInternalTx<K, V>> txIdMap = transactionMap(tx);
 
         // Start clean.
         txContextReset();
@@ -422,7 +422,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
             return null;
         }
 
-        IgniteTxEx<K, V> t;
+        IgniteInternalTx<K, V> t;
 
         if ((t = txIdMap.putIfAbsent(tx.xidVersion(), tx)) == null) {
             // Add both, explicit and implicit transactions.
@@ -517,10 +517,10 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @return Future that will be completed when all ongoing transactions are finished.
      */
     public IgniteInternalFuture<Boolean> finishTxs(long topVer) {
-        GridCompoundFuture<IgniteTxEx, Boolean> res =
+        GridCompoundFuture<IgniteInternalTx, Boolean> res =
             new GridCompoundFuture<>(context().kernalContext(),
-                new IgniteReducer<IgniteTxEx, Boolean>() {
-                    @Override public boolean collect(IgniteTxEx e) {
+                new IgniteReducer<IgniteInternalTx, Boolean>() {
+                    @Override public boolean collect(IgniteInternalTx e) {
                         return true;
                     }
 
@@ -529,7 +529,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
                     }
                 });
 
-        for (IgniteTxEx<K, V> tx : txs()) {
+        for (IgniteInternalTx<K, V> tx : txs()) {
             // Must wait for all transactions, even for DHT local and DHT remote since preloading may acquire
             // values pending to be overwritten by prepared transaction.
 
@@ -564,7 +564,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @param tx Started transaction.
      * @return {@code True} if transaction is not in completed set.
      */
-    public boolean onStarted(IgniteTxEx<K, V> tx) {
+    public boolean onStarted(IgniteInternalTx<K, V> tx) {
         assert tx.state() == ACTIVE || tx.isRollbackOnly() : "Invalid transaction state [locId=" + cctx.localNodeId() +
             ", tx=" + tx + ']';
 
@@ -590,7 +590,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @return Near version.
      */
     @Nullable public GridCacheVersion nearVersion(GridCacheVersion dhtVer) {
-        IgniteTxEx<K, V> tx = idMap.get(dhtVer);
+        IgniteInternalTx<K, V> tx = idMap.get(dhtVer);
 
         if (tx != null)
             return tx.nearXidVersion();
@@ -616,7 +616,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @param ver Alternate version.
      * @param tx Transaction.
      */
-    public void addAlternateVersion(GridCacheVersion ver, IgniteTxEx<K, V> tx) {
+    public void addAlternateVersion(GridCacheVersion ver, IgniteInternalTx<K, V> tx) {
         if (idMap.putIfAbsent(ver, tx) == null)
             if (log.isDebugEnabled())
                 log.debug("Registered alternate transaction version [ver=" + ver + ", tx=" + tx + ']');
@@ -627,7 +627,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      */
     @SuppressWarnings({"unchecked"})
     @Nullable public <T> T localTx() {
-        IgniteTxEx<K, V> tx = tx();
+        IgniteInternalTx<K, V> tx = tx();
 
         return tx != null && tx.local() ? (T)tx : null;
     }
@@ -637,7 +637,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      */
     @SuppressWarnings({"unchecked"})
     public <T> T threadLocalTx() {
-        IgniteTxEx<K, V> tx = tx(Thread.currentThread().getId());
+        IgniteInternalTx<K, V> tx = tx(Thread.currentThread().getId());
 
         return tx != null && tx.local() && (!tx.dht() || tx.colocated()) && !tx.implicit() ? (T)tx : null;
     }
@@ -647,7 +647,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      */
     @SuppressWarnings({"unchecked", "RedundantCast"})
     public <T> T tx() {
-        IgniteTxEx<K, V> tx = txContext();
+        IgniteInternalTx<K, V> tx = txContext();
 
         return tx != null ? (T)tx : (T)tx(Thread.currentThread().getId());
     }
@@ -655,8 +655,8 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
     /**
      * @return Local transaction.
      */
-    @Nullable public IgniteTxEx<K, V> localTxx() {
-        IgniteTxEx<K, V> tx = txx();
+    @Nullable public IgniteInternalTx<K, V> localTxx() {
+        IgniteInternalTx<K, V> tx = txx();
 
         return tx != null && tx.local() ? tx : null;
     }
@@ -665,15 +665,15 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @return Transaction for current thread.
      */
     @SuppressWarnings({"unchecked"})
-    public IgniteTxEx<K, V> txx() {
+    public IgniteInternalTx<K, V> txx() {
         return tx();
     }
 
     /**
      * @return User transaction for current thread.
      */
-    @Nullable public IgniteTxEx userTx() {
-        IgniteTxEx<K, V> tx = txContext();
+    @Nullable public IgniteInternalTx userTx() {
+        IgniteInternalTx<K, V> tx = txContext();
 
         if (tx != null && tx.user() && tx.state() == ACTIVE)
             return tx;
@@ -712,7 +712,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @return Transaction with given ID.
      */
     @SuppressWarnings({"unchecked"})
-    @Nullable public <T extends IgniteTxEx<K, V>> T tx(GridCacheVersion txId) {
+    @Nullable public <T extends IgniteInternalTx<K, V>> T tx(GridCacheVersion txId) {
         return (T)idMap.get(txId);
     }
 
@@ -721,7 +721,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @return Transaction with given ID.
      */
     @SuppressWarnings({"unchecked"})
-    @Nullable public <T extends IgniteTxEx<K, V>> T nearTx(GridCacheVersion txId) {
+    @Nullable public <T extends IgniteInternalTx<K, V>> T nearTx(GridCacheVersion txId) {
         return (T)nearIdMap.get(txId);
     }
 
@@ -729,7 +729,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @param txId Transaction ID.
      * @return Transaction with given ID.
      */
-    @Nullable public IgniteTxEx<K, V> txx(GridCacheVersion txId) {
+    @Nullable public IgniteInternalTx<K, V> txx(GridCacheVersion txId) {
         return idMap.get(txId);
     }
 
@@ -739,7 +739,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @param tx Transaction to prepare.
      * @throws IgniteCheckedException If preparation failed.
      */
-    public void prepareTx(IgniteTxEx<K, V> tx) throws IgniteCheckedException {
+    public void prepareTx(IgniteInternalTx<K, V> tx) throws IgniteCheckedException {
         if (tx.state() == MARKED_ROLLBACK) {
             if (tx.timedOut())
                 throw new IgniteTxTimeoutCheckedException("Transaction timed out: " + this);
@@ -758,8 +758,8 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
         // Clean up committed transactions queue.
         if (tx.pessimistic()) {
             if (tx.enforceSerializable() && txSerializableEnabled) {
-                for (Iterator<IgniteTxEx<K, V>> it = committedQ.iterator(); it.hasNext();) {
-                    IgniteTxEx<K, V> committedTx = it.next();
+                for (Iterator<IgniteInternalTx<K, V>> it = committedQ.iterator(); it.hasNext();) {
+                    IgniteInternalTx<K, V> committedTx = it.next();
 
                     assert committedTx != tx;
 
@@ -788,8 +788,8 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
             // Check that our read set does not intersect with write set
             // of all transactions that completed their write phase
             // while our transaction was in read phase.
-            for (Iterator<IgniteTxEx<K, V>> it = committedQ.iterator(); it.hasNext();) {
-                IgniteTxEx<K, V> committedTx = it.next();
+            for (Iterator<IgniteInternalTx<K, V>> it = committedQ.iterator(); it.hasNext();) {
+                IgniteInternalTx<K, V> committedTx = it.next();
 
                 assert committedTx != tx;
 
@@ -819,8 +819,8 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
 
             // Check that our read and write sets do not intersect with write
             // sets of all active transactions.
-            for (Iterator<IgniteTxEx<K, V>> iter = prepareQ.iterator(); iter.hasNext();) {
-                IgniteTxEx<K, V> prepareTx = iter.next();
+            for (Iterator<IgniteInternalTx<K, V>> iter = prepareQ.iterator(); iter.hasNext();) {
+                IgniteInternalTx<K, V> prepareTx = iter.next();
 
                 if (prepareTx == tx)
                     // Skip yourself.
@@ -879,7 +879,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @param tx Transaction to check.
      * @return {@code True} if transaction can be discarded.
      */
-    private boolean isSafeToForget(IgniteTxEx<K, V> tx) {
+    private boolean isSafeToForget(IgniteInternalTx<K, V> tx) {
         Map.Entry<GridCacheVersion, AtomicInt> e = startVerCnts.firstEntry();
 
         if (e == null)
@@ -895,7 +895,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      *
      * @param tx Cache transaction.
      */
-    private void decrementStartVersionCount(IgniteTxEx<K, V> tx) {
+    private void decrementStartVersionCount(IgniteInternalTx<K, V> tx) {
         AtomicInt cnt = startVerCnts.get(tx.startVersion());
 
         assert cnt != null : "Failed to find start version count for transaction [startVerCnts=" + startVerCnts +
@@ -912,7 +912,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
     /**
      * @param tx Transaction.
      */
-    private void removeObsolete(IgniteTxEx<K, V> tx) {
+    private void removeObsolete(IgniteInternalTx<K, V> tx) {
         Collection<IgniteTxEntry<K, V>> entries = (tx.local() && !tx.dht()) ? tx.allEntries() : tx.writeEntries();
 
         for (IgniteTxEntry<K, V> entry : entries) {
@@ -985,7 +985,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
     /**
      * @param tx Tx to remove.
      */
-    public void removeCommittedTx(IgniteTxEx<K, V> tx) {
+    public void removeCommittedTx(IgniteInternalTx<K, V> tx) {
         committedVers.remove(tx.xidVersion());
     }
 
@@ -993,7 +993,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @param tx Committed transaction.
      * @return If transaction was not already present in committed set.
      */
-    public boolean addCommittedTx(IgniteTxEx<K, V> tx) {
+    public boolean addCommittedTx(IgniteInternalTx<K, V> tx) {
         return addCommittedTx(tx.xidVersion(), tx.nearXidVersion());
     }
 
@@ -1001,7 +1001,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @param tx Committed transaction.
      * @return If transaction was not already present in committed set.
      */
-    public boolean addRolledbackTx(IgniteTxEx<K, V> tx) {
+    public boolean addRolledbackTx(IgniteInternalTx<K, V> tx) {
         return addRolledbackTx(tx.xidVersion());
     }
 
@@ -1054,7 +1054,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
     /**
      * @param tx Transaction.
      */
-    private void processCompletedEntries(IgniteTxEx<K, V> tx) {
+    private void processCompletedEntries(IgniteInternalTx<K, V> tx) {
         if (tx.needsCompletedVersions()) {
             GridCacheVersion min = minVersion(tx.readEntries(), tx.xidVersion(), tx);
 
@@ -1133,7 +1133,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @return Minimal available version.
      */
     private GridCacheVersion minVersion(Iterable<IgniteTxEntry<K, V>> entries, GridCacheVersion min,
-        IgniteTxEx<K, V> tx) {
+        IgniteInternalTx<K, V> tx) {
         for (IgniteTxEntry<K, V> txEntry : entries) {
             GridCacheEntryEx<K, V> cached = txEntry.cached();
 
@@ -1155,7 +1155,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      *
      * @param tx Transaction to commit.
      */
-    public void commitTx(IgniteTxEx<K, V> tx) {
+    public void commitTx(IgniteInternalTx<K, V> tx) {
         assert tx != null;
         assert tx.state() == COMMITTING : "Invalid transaction state for commit from tm [state=" + tx.state() +
             ", expected=COMMITTING, tx=" + tx + ']';
@@ -1184,7 +1184,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
                 committedVers.firstx() + ", lastVer=" + committedVers.lastx() + ", tx=" + tx.xid() + ']');
         }
 
-        ConcurrentMap<GridCacheVersion, IgniteTxEx<K, V>> txIdMap = transactionMap(tx);
+        ConcurrentMap<GridCacheVersion, IgniteInternalTx<K, V>> txIdMap = transactionMap(tx);
 
         if (txIdMap.remove(tx.xidVersion(), tx)) {
             // 2. Must process completed entries before unlocking!
@@ -1281,7 +1281,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      *
      * @param tx Transaction to rollback.
      */
-    public void rollbackTx(IgniteTxEx<K, V> tx) {
+    public void rollbackTx(IgniteInternalTx<K, V> tx) {
         assert tx != null;
 
         if (log.isDebugEnabled())
@@ -1290,7 +1290,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
         // 1. Record transaction version to avoid duplicates.
         addRolledbackTx(tx);
 
-        ConcurrentMap<GridCacheVersion, IgniteTxEx<K, V>> txIdMap = transactionMap(tx);
+        ConcurrentMap<GridCacheVersion, IgniteInternalTx<K, V>> txIdMap = transactionMap(tx);
 
         if (txIdMap.remove(tx.xidVersion(), tx)) {
             // 2. Unlock write resources.
@@ -1349,13 +1349,13 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      *
      * @param tx Tx to uncommit.
      */
-    public void uncommitTx(IgniteTxEx<K, V> tx) {
+    public void uncommitTx(IgniteInternalTx<K, V> tx) {
         assert tx != null;
 
         if (log.isDebugEnabled())
             log.debug("Uncommiting from TM: " + tx);
 
-        ConcurrentMap<GridCacheVersion, IgniteTxEx<K, V>> txIdMap = transactionMap(tx);
+        ConcurrentMap<GridCacheVersion, IgniteInternalTx<K, V>> txIdMap = transactionMap(tx);
 
         if (txIdMap.remove(tx.xidVersion(), tx)) {
             // 1. Unlock write resources.
@@ -1401,14 +1401,14 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @param tx Transaction.
      * @return Transaction map.
      */
-    private ConcurrentMap<GridCacheVersion, IgniteTxEx<K, V>> transactionMap(IgniteTxEx<K, V> tx) {
+    private ConcurrentMap<GridCacheVersion, IgniteInternalTx<K, V>> transactionMap(IgniteInternalTx<K, V> tx) {
         return (tx.near() && !tx.local()) ? nearIdMap : idMap;
     }
 
     /**
      * @param tx Transaction to notify evictions for.
      */
-    private void notifyEvitions(IgniteTxEx<K, V> tx) {
+    private void notifyEvitions(IgniteInternalTx<K, V> tx) {
         if (tx.internal() && !tx.groupLock())
             return;
 
@@ -1514,7 +1514,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @return {@code True} if all keys were locked.
      * @throws IgniteCheckedException If lock has been cancelled.
      */
-    private boolean lockMultiple(IgniteTxEx<K, V> tx, Iterable<IgniteTxEntry<K, V>> entries)
+    private boolean lockMultiple(IgniteInternalTx<K, V> tx, Iterable<IgniteTxEntry<K, V>> entries)
         throws IgniteCheckedException {
         assert tx.optimistic();
 
@@ -1590,7 +1590,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @param txx Transaction.
      */
     @SuppressWarnings("unchecked")
-    private void unlockGroupLocks(IgniteTxEx txx) {
+    private void unlockGroupLocks(IgniteInternalTx txx) {
         IgniteTxKey grpLockKey = txx.groupLockKey();
 
         assert grpLockKey != null;
@@ -1633,7 +1633,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @param tx Owning transaction.
      * @param entries Entries to unlock.
      */
-    private void unlockMultiple(IgniteTxEx<K, V> tx, Iterable<IgniteTxEntry<K, V>> entries) {
+    private void unlockMultiple(IgniteInternalTx<K, V> tx, Iterable<IgniteTxEntry<K, V>> entries) {
         for (IgniteTxEntry<K, V> txEntry : entries) {
             GridCacheContext<K, V> cacheCtx = txEntry.context();
 
@@ -1693,7 +1693,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @param newState New state.
      * @param tx Cache transaction.
      */
-    public void onTxStateChange(@Nullable IgniteTxState prevState, IgniteTxState newState, IgniteTxEx tx) {
+    public void onTxStateChange(@Nullable IgniteTxState prevState, IgniteTxState newState, IgniteInternalTx tx) {
         // Notify synchronizations.
         for (IgniteTxSynchronization s : syncs)
             s.onStateChanged(prevState, newState, tx.proxy());
@@ -1702,7 +1702,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
     /**
      * @param tx Committing transaction.
      */
-    public void txContext(IgniteTxEx tx) {
+    public void txContext(IgniteInternalTx tx) {
         threadCtx.set(tx);
     }
 
@@ -1710,7 +1710,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @return Currently committing transaction.
      */
     @SuppressWarnings({"unchecked"})
-    private IgniteTxEx<K, V> txContext() {
+    private IgniteInternalTx<K, V> txContext() {
         return threadCtx.get();
     }
 
@@ -1723,7 +1723,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @return Transaction version from transaction context.
      */
     @Nullable public GridCacheVersion txContextVersion() {
-        IgniteTxEx<K, V> tx = txContext();
+        IgniteInternalTx<K, V> tx = txContext();
 
         return tx == null ? null : tx.xidVersion();
     }
@@ -1738,7 +1738,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
     /**
      * @return All transactions.
      */
-    public Collection<IgniteTxEx<K, V>> txs() {
+    public Collection<IgniteInternalTx<K, V>> txs() {
         return F.concat(false, idMap.values(), nearIdMap.values());
     }
 
@@ -1766,7 +1766,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
     public boolean txsPreparedOrCommitted(GridCacheVersion nearVer, int txNum) {
         Collection<GridCacheVersion> processedVers = null;
 
-        for (IgniteTxEx<K, V> tx : txs()) {
+        for (IgniteInternalTx<K, V> tx : txs()) {
             if (nearVer.equals(tx.nearXidVersion())) {
                 IgniteTxState state = tx.state();
 
@@ -1828,7 +1828,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      *
      * @param tx Committed transaction to add.
      */
-    private void addPessimisticRecovery(IgniteTxEx<K, V> tx) {
+    private void addPessimisticRecovery(IgniteInternalTx<K, V> tx) {
         if (pessimisticRecoveryBuf == null)
             return;
 
@@ -1860,13 +1860,13 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @param nearXidVer Near tx ID.
      * @return Near local or colocated local transaction.
      */
-    @Nullable public IgniteTxEx<K, V> localTxForRecovery(GridCacheVersion nearXidVer, boolean markFinalizing) {
+    @Nullable public IgniteInternalTx<K, V> localTxForRecovery(GridCacheVersion nearXidVer, boolean markFinalizing) {
         // First check if we have near transaction with this ID.
-        IgniteTxEx<K, V> tx = idMap.get(nearXidVer);
+        IgniteInternalTx<K, V> tx = idMap.get(nearXidVer);
 
         if (tx == null) {
             // Check all local transactions and mark them as waiting for recovery to prevent finish race.
-            for (IgniteTxEx<K, V> txEx : idMap.values()) {
+            for (IgniteInternalTx<K, V> txEx : idMap.values()) {
                 if (nearXidVer.equals(txEx.nearXidVersion())) {
                     if (!markFinalizing || !txEx.markFinalizing(RECOVERY_WAIT))
                         tx = txEx;
@@ -1888,7 +1888,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @param tx Transaction.
      * @param commit Whether transaction should be committed or rolled back.
      */
-    public void finishOptimisticTxOnRecovery(final IgniteTxEx<K, V> tx, boolean commit) {
+    public void finishOptimisticTxOnRecovery(final IgniteInternalTx<K, V> tx, boolean commit) {
         if (log.isDebugEnabled())
             log.debug("Finishing prepared transaction [tx=" + tx + ", commit=" + commit + ']');
 
@@ -1918,7 +1918,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @param tx Transaction to finish.
      * @param commitInfo Commit information.
      */
-    public void finishPessimisticTxOnRecovery(final IgniteTxEx<K, V> tx, GridCacheCommittedTxInfo<K, V> commitInfo) {
+    public void finishPessimisticTxOnRecovery(final IgniteInternalTx<K, V> tx, GridCacheCommittedTxInfo<K, V> commitInfo) {
         if (!tx.markFinalizing(RECOVERY_FINISH)) {
             if (log.isDebugEnabled())
                 log.debug("Will not try to finish pessimistic transaction (could not mark as finalizing): " + tx);
@@ -1982,7 +1982,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      */
     public IgniteInternalFuture<GridCacheCommittedTxInfo<K, V>> checkPessimisticTxCommitted(GridCachePessimisticCheckCommittedTxRequest req) {
         // First check if we have near transaction with this ID.
-        IgniteTxEx<K, V> tx = localTxForRecovery(req.nearXidVersion(), !req.nearOnlyCheck());
+        IgniteInternalTx<K, V> tx = localTxForRecovery(req.nearXidVersion(), !req.nearOnlyCheck());
 
         // Either we found near transaction or one of transactions is being committed by user.
         // Wait for it and send reply.
@@ -1992,10 +1992,10 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
             if (log.isDebugEnabled())
                 log.debug("Found active near transaction, will wait for completion [req=" + req + ", tx=" + tx + ']');
 
-            final IgniteTxEx<K, V> tx0 = tx;
+            final IgniteInternalTx<K, V> tx0 = tx;
 
-            return tx.finishFuture().chain(new C1<IgniteInternalFuture<IgniteTxEx>, GridCacheCommittedTxInfo<K, V>>() {
-                @Override public GridCacheCommittedTxInfo<K, V> apply(IgniteInternalFuture<IgniteTxEx> txFut) {
+            return tx.finishFuture().chain(new C1<IgniteInternalFuture<IgniteInternalTx>, GridCacheCommittedTxInfo<K, V>>() {
+                @Override public GridCacheCommittedTxInfo<K, V> apply(IgniteInternalFuture<IgniteInternalTx> txFut) {
                     GridCacheCommittedTxInfo<K, V> info = null;
 
                     if (tx0.state() == COMMITTED)
@@ -2048,7 +2048,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
                     log.debug("Processing node failed event [locNodeId=" + cctx.localNodeId() +
                         ", failedNodeId=" + evtNodeId + ']');
 
-                for (IgniteTxEx<K, V> tx : txs()) {
+                for (IgniteInternalTx<K, V> tx : txs()) {
                     if ((tx.near() && !tx.local()) || (tx.storeUsed() && tx.masterNodeIds().contains(evtNodeId))) {
                         // Invalidate transactions.
                         salvageTx(tx, false, RECOVERY_FINISH);
@@ -2095,7 +2095,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
          *
          * @param tx Transaction.
          */
-        private void commitIfPrepared(IgniteTxEx<K, V> tx) {
+        private void commitIfPrepared(IgniteInternalTx<K, V> tx) {
             assert tx instanceof GridDhtTxLocal || tx instanceof GridDhtTxRemote  : tx;
             assert !F.isEmpty(tx.transactionNodes());
             assert tx.nearXidVersion() != null;
@@ -2117,7 +2117,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
          *
          * @param tx Transaction.
          */
-        private void commitIfRemotelyCommitted(IgniteTxEx<K, V> tx) {
+        private void commitIfRemotelyCommitted(IgniteInternalTx<K, V> tx) {
             assert tx instanceof GridDhtTxLocal || tx instanceof GridDhtTxRemote : tx;
 
             GridCachePessimisticCheckCommittedTxFuture<K, V> fut = new GridCachePessimisticCheckCommittedTxFuture<>(
@@ -2191,22 +2191,22 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
     /**
      * Commit listener. Checks if commit succeeded and rollbacks if case of error.
      */
-    private class CommitListener implements CI1<IgniteInternalFuture<IgniteTxEx>> {
+    private class CommitListener implements CI1<IgniteInternalFuture<IgniteInternalTx>> {
         /** */
         private static final long serialVersionUID = 0L;
 
         /** Transaction. */
-        private final IgniteTxEx<K, V> tx;
+        private final IgniteInternalTx<K, V> tx;
 
         /**
          * @param tx Transaction.
          */
-        private CommitListener(IgniteTxEx<K, V> tx) {
+        private CommitListener(IgniteInternalTx<K, V> tx) {
             this.tx = tx;
         }
 
         /** {@inheritDoc} */
-        @Override public void apply(IgniteInternalFuture<IgniteTxEx> t) {
+        @Override public void apply(IgniteInternalFuture<IgniteInternalTx> t) {
             try {
                 t.get();
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxProxyImpl.java
index 8d14acf..362d2e0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxProxyImpl.java
@@ -39,7 +39,7 @@ public class IgniteTxProxyImpl<K, V> implements IgniteTxProxy, Externalizable {
 
     /** Wrapped transaction. */
     @GridToStringInclude
-    private IgniteTxEx<K, V> tx;
+    private IgniteInternalTx<K, V> tx;
 
     /** Gateway. */
     @GridToStringExclude
@@ -63,7 +63,7 @@ public class IgniteTxProxyImpl<K, V> implements IgniteTxProxy, Externalizable {
      * @param cctx Shared context.
      * @param async Async flag.
      */
-    public IgniteTxProxyImpl(IgniteTxEx<K, V> tx, GridCacheSharedContext<K, V> cctx, boolean async) {
+    public IgniteTxProxyImpl(IgniteInternalTx<K, V> tx, GridCacheSharedContext<K, V> cctx, boolean async) {
         assert tx != null;
         assert cctx != null;
 
@@ -234,7 +234,7 @@ public class IgniteTxProxyImpl<K, V> implements IgniteTxProxy, Externalizable {
         enter();
 
         try {
-            IgniteInternalFuture<IgniteTxEx> commitFut = cctx.commitTxAsync(tx);
+            IgniteInternalFuture<IgniteInternalTx> commitFut = cctx.commitTxAsync(tx);
 
             if (async)
                 saveFuture(commitFut);
@@ -311,9 +311,9 @@ public class IgniteTxProxyImpl<K, V> implements IgniteTxProxy, Externalizable {
     /**
      * @param fut Internal future.
      */
-    private void saveFuture(IgniteInternalFuture<IgniteTxEx> fut) {
-        IgniteInternalFuture<IgniteTx> fut0 = fut.chain(new CX1<IgniteInternalFuture<IgniteTxEx>, IgniteTx>() {
-            @Override public IgniteTx applyx(IgniteInternalFuture<IgniteTxEx> fut) throws IgniteCheckedException {
+    private void saveFuture(IgniteInternalFuture<IgniteInternalTx> fut) {
+        IgniteInternalFuture<IgniteTx> fut0 = fut.chain(new CX1<IgniteInternalFuture<IgniteInternalTx>, IgniteTx>() {
+            @Override public IgniteTx applyx(IgniteInternalFuture<IgniteInternalTx> fut) throws IgniteCheckedException {
                 return fut.get().proxy();
             }
         });
@@ -328,7 +328,7 @@ public class IgniteTxProxyImpl<K, V> implements IgniteTxProxy, Externalizable {
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        tx = (IgniteTxEx<K, V>)in.readObject();
+        tx = (IgniteInternalTx<K, V>)in.readObject();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteEx.java
index ff9a92c..adbccea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteEx.java
@@ -24,7 +24,7 @@ import java.util.*;
 /**
  * Local transaction API.
  */
-public interface IgniteTxRemoteEx<K, V> extends IgniteTxEx<K, V> {
+public interface IgniteTxRemoteEx<K, V> extends IgniteInternalTx<K, V> {
     /**
      * @return Remote thread ID.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsDataManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsDataManager.java
index 53e4ca2..bd9802a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsDataManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsDataManager.java
@@ -30,7 +30,6 @@ import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.thread.*;
-import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.managers.eventstorage.*;
 import org.apache.ignite.internal.processors.dataload.*;
@@ -687,7 +686,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
                         // Need to check if block is partially written.
                         // If so, must update it in pessimistic transaction.
                         if (block.length != fileInfo.blockSize()) {
-                            try (IgniteTxEx tx = dataCachePrj.txStartEx(PESSIMISTIC, REPEATABLE_READ)) {
+                            try (IgniteInternalTx tx = dataCachePrj.txStartEx(PESSIMISTIC, REPEATABLE_READ)) {
                                 Map<GridGgfsBlockKey, byte[]> vals = dataCachePrj.getAll(F.asList(colocatedKey, key));
 
                                 byte[] val = vals.get(colocatedKey);
@@ -1130,7 +1129,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
         GridGgfsBlockKey key = new GridGgfsBlockKey(colocatedKey.getFileId(), null,
             colocatedKey.evictExclude(), colocatedKey.getBlockId());
 
-        try (IgniteTxEx tx = dataCachePrj.txStartEx(PESSIMISTIC, REPEATABLE_READ)) {
+        try (IgniteInternalTx tx = dataCachePrj.txStartEx(PESSIMISTIC, REPEATABLE_READ)) {
             // Lock keys.
             Map<GridGgfsBlockKey, byte[]> vals = dataCachePrj.getAll(F.asList(colocatedKey, key));
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsMetaManager.java
index 3f070b9..ce33c60 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsMetaManager.java
@@ -408,7 +408,7 @@ public class GridGgfsMetaManager extends GridGgfsManager {
                 assert validTxState(false);
                 assert fileId != null;
 
-                IgniteTxEx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
+                IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
 
                 try {
                     // Lock file ID for this transaction.
@@ -491,7 +491,7 @@ public class GridGgfsMetaManager extends GridGgfsManager {
 
                 IgniteUuid fileId = info.id();
 
-                IgniteTxEx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
+                IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
 
                 try {
                     // Lock file ID for this transaction.
@@ -692,7 +692,7 @@ public class GridGgfsMetaManager extends GridGgfsManager {
 
                 IgniteUuid res = null;
 
-                IgniteTxEx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
+                IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
 
                 try {
                     res = putIfAbsentNonTx(parentId, fileName, newFileInfo);
@@ -781,7 +781,7 @@ public class GridGgfsMetaManager extends GridGgfsManager {
             try {
                 assert validTxState(false);
 
-                IgniteTxEx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
+                IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
 
                 try {
                     moveNonTx(fileId, srcFileName, srcParentId, destFileName, destParentId);
@@ -901,7 +901,7 @@ public class GridGgfsMetaManager extends GridGgfsManager {
             try {
                 assert validTxState(false);
 
-                IgniteTxEx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
+                IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
 
                 try {
                     if (parentId != null)
@@ -1015,7 +1015,7 @@ public class GridGgfsMetaManager extends GridGgfsManager {
             try {
                 assert validTxState(false);
 
-                IgniteTxEx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
+                IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
 
                 try {
                     if (parentId == null)
@@ -1136,7 +1136,7 @@ public class GridGgfsMetaManager extends GridGgfsManager {
                 assert listing != null;
                 assert validTxState(false);
 
-                IgniteTxEx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
+                IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
 
                 try {
                     Collection<IgniteUuid> res = new HashSet<>();
@@ -1221,7 +1221,7 @@ public class GridGgfsMetaManager extends GridGgfsManager {
             try {
                 assert validTxState(false);
 
-                IgniteTxEx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
+                IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
 
                 try {
                     boolean res = false;
@@ -1394,7 +1394,7 @@ public class GridGgfsMetaManager extends GridGgfsManager {
             try {
                 assert validTxState(false);
 
-                IgniteTxEx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
+                IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
 
                 try {
                     GridGgfsFileInfo info = updatePropertiesNonTx(parentId, fileId, fileName, props);
@@ -1464,7 +1464,7 @@ public class GridGgfsMetaManager extends GridGgfsManager {
                 if (log.isDebugEnabled())
                     log.debug("Update file info [fileId=" + fileId + ", c=" + c + ']');
 
-                IgniteTxEx tx = metaCache.isLockedByThread(fileId) ? null : metaCache.txStartEx(PESSIMISTIC,
+                IgniteInternalTx tx = metaCache.isLockedByThread(fileId) ? null : metaCache.txStartEx(PESSIMISTIC,
                     REPEATABLE_READ);
 
                 try {
@@ -1527,7 +1527,7 @@ public class GridGgfsMetaManager extends GridGgfsManager {
             try {
                 validTxState(false);
 
-                IgniteTxEx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
+                IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
 
                 try {
                     Object prev = val != null ? metaCache.put(sampling, val) : metaCache.remove(sampling);
@@ -2374,7 +2374,7 @@ public class GridGgfsMetaManager extends GridGgfsManager {
                 pathIds.add(fileIds(path));
 
             // Start pessimistic.
-            IgniteTxEx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
+            IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
 
             try {
                 // Lock the very first existing parents and possibly the leaf as well.
@@ -2600,7 +2600,7 @@ public class GridGgfsMetaManager extends GridGgfsManager {
                 assert validTxState(false);
 
                 // Start pessimistic transaction.
-                IgniteTxEx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
+                IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
 
                 try {
                     Map<IgniteUuid, GridGgfsFileInfo> infoMap = lockIds(fileId, parentId);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
index cabbc29..4a7fd26 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
@@ -26,7 +26,6 @@ import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
-import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.processors.license.*;
 import org.apache.ignite.internal.processors.rest.*;
 import org.apache.ignite.internal.processors.rest.handlers.*;
@@ -469,7 +468,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
 
         return ctx.closure().callLocalSafe(new Callable<Object>() {
             @Override public Object call() throws Exception {
-                try (IgniteTxEx tx = cache.txStartEx(PESSIMISTIC, REPEATABLE_READ)) {
+                try (IgniteInternalTx tx = cache.txStartEx(PESSIMISTIC, REPEATABLE_READ)) {
                     Object curVal = cache.get(key);
 
                     if (curVal == null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d89ef5bd/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index dbada06..9ac1571 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -32,7 +32,6 @@ import org.apache.ignite.lang.*;
 import org.apache.ignite.managed.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.thread.*;
-import org.apache.ignite.transactions.*;
 import org.apache.ignite.cache.query.CacheContinuousQueryEntry;
 import org.apache.ignite.internal.managers.eventstorage.*;
 import org.apache.ignite.internal.processors.cache.query.continuous.*;
@@ -629,7 +628,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
         Object affKey = cfg.getAffinityKey();
 
         while (true) {
-            try (IgniteTxEx tx = cache.txStartEx(PESSIMISTIC, REPEATABLE_READ)) {
+            try (IgniteInternalTx tx = cache.txStartEx(PESSIMISTIC, REPEATABLE_READ)) {
                 GridServiceAssignmentsKey key = new GridServiceAssignmentsKey(cfg.getName());
 
                 GridServiceAssignments oldAssigns = (GridServiceAssignments)cache.get(key);