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:58 UTC

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

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);