http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxMetricsAdapter.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxMetricsAdapter.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxMetricsAdapter.java
deleted file mode 100644
index 3669cf2..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxMetricsAdapter.java
+++ /dev/null
@@ -1,119 +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.internal.util.typedef.internal.*;
-import org.apache.ignite.transactions.*;
-
-import java.io.*;
-
-/**
- * Tx metrics adapter.
- */
-public class IgniteTxMetricsAdapter implements IgniteTxMetrics, Externalizable 
{
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Number of transaction commits. */
-    private volatile int txCommits;
-
-    /** Number of transaction rollbacks. */
-    private volatile int txRollbacks;
-
-    /** Last commit time. */
-    private volatile long commitTime;
-
-    /** Last rollback time. */
-    private volatile long rollbackTime;
-
-    /**
-     *
-     */
-    public IgniteTxMetricsAdapter() {
-
-    }
-
-    /**
-     * @param m Transaction metrics to copy.
-     */
-    public IgniteTxMetricsAdapter(IgniteTxMetrics m) {
-        commitTime = m.commitTime();
-        rollbackTime = m.rollbackTime();
-        txCommits = m.txCommits();
-        txRollbacks = m.txRollbacks();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long commitTime() {
-        return commitTime;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long rollbackTime() {
-        return rollbackTime;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int txCommits() {
-        return txCommits;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int txRollbacks() {
-        return txRollbacks;
-    }
-
-    /**
-     * Transaction commit callback.
-     */
-    public void onTxCommit() {
-        commitTime = U.currentTimeMillis();
-
-        txCommits++;
-    }
-
-    /**
-     * Transaction rollback callback.
-     */
-    public void onTxRollback() {
-        rollbackTime = U.currentTimeMillis();
-
-        txRollbacks++;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeLong(commitTime);
-        out.writeLong(rollbackTime);
-        out.writeInt(txCommits);
-        out.writeInt(txRollbacks);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, 
ClassNotFoundException {
-        commitTime = in.readLong();
-        rollbackTime = in.readLong();
-        txCommits = in.readInt();
-        txRollbacks = in.readInt();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(IgniteTxMetricsAdapter.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxProxy.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxProxy.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxProxy.java
deleted file mode 100644
index 5ad6542..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxProxy.java
+++ /dev/null
@@ -1,27 +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.transactions.*;
-
-/**
- * Marker interface for transaction proxy.
- */
-public interface IgniteTxProxy extends IgniteTx {
-    // No-op.
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/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
deleted file mode 100644
index 480ccb4..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxProxyImpl.java
+++ /dev/null
@@ -1,345 +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.internal.*;
-import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.internal.util.future.*;
-import org.apache.ignite.internal.util.tostring.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.transactions.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Cache transaction proxy.
- */
-public class IgniteTxProxyImpl<K, V> implements IgniteTxProxy, Externalizable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Wrapped transaction. */
-    @GridToStringInclude
-    private IgniteInternalTx<K, V> tx;
-
-    /** Gateway. */
-    @GridToStringExclude
-    private GridCacheSharedContext<K, V> cctx;
-
-    /** Async flag. */
-    private boolean async;
-
-    /** Async call result. */
-    private IgniteFuture asyncRes;
-
-    /**
-     * Empty constructor required for {@link Externalizable}.
-     */
-    public IgniteTxProxyImpl() {
-        // No-op.
-    }
-
-    /**
-     * @param tx Transaction.
-     * @param cctx Shared context.
-     * @param async Async flag.
-     */
-    public IgniteTxProxyImpl(IgniteInternalTx<K, V> tx, 
GridCacheSharedContext<K, V> cctx, boolean async) {
-        assert tx != null;
-        assert cctx != null;
-
-        this.tx = tx;
-        this.cctx = cctx;
-        this.async = async;
-    }
-
-    /**
-     * @return Transaction.
-     */
-    public IgniteInternalTx<K, V> tx() {
-        return tx;
-    }
-
-    /**
-     * Enters a call.
-     */
-    private void enter() {
-        if (cctx.deploymentEnabled())
-            cctx.deploy().onEnter();
-
-        try {
-            cctx.kernalContext().gateway().readLock();
-        }
-        catch (IllegalStateException e) {
-            throw e;
-        }
-        catch (RuntimeException | Error e) {
-            cctx.kernalContext().gateway().readUnlock();
-
-            throw e;
-        }
-    }
-
-    /**
-     * Leaves a call.
-     */
-    private void leave() {
-        try {
-            CU.unwindEvicts(cctx);
-        }
-        finally {
-            cctx.kernalContext().gateway().readUnlock();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteUuid xid() {
-        return tx.xid();
-    }
-
-    /** {@inheritDoc} */
-    @Override public UUID nodeId() {
-        if (async)
-            save(tx.nodeId());
-
-        return tx.nodeId();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long threadId() {
-        if (async)
-            save(tx.threadId());
-
-        return tx.threadId();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long startTime() {
-        if (async)
-            save(tx.startTime());
-
-        return tx.startTime();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteTxIsolation isolation() {
-        if (async)
-            save(tx.isolation());
-
-        return tx.isolation();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteTxConcurrency concurrency() {
-        if (async)
-            save(tx.concurrency());
-
-        return tx.concurrency();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isInvalidate() {
-        if (async)
-            save(tx.isInvalidate());
-
-        return tx.isInvalidate();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean implicit() {
-        if (async)
-            save(tx.implicit());
-
-        return tx.implicit();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long timeout() {
-        if (async)
-            save(tx.timeout());
-
-        return tx.timeout();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteTxState state() {
-        if (async)
-            save(tx.state());
-
-        return tx.state();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long timeout(long timeout) {
-        return tx.timeout(timeout);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteAsyncSupport withAsync() {
-        return new IgniteTxProxyImpl<>(tx, cctx, true);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isAsync() {
-        return async;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public <R> IgniteFuture<R> future() {
-        return asyncRes;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean setRollbackOnly() {
-        enter();
-
-        try {
-            return tx.setRollbackOnly();
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isRollbackOnly() {
-        enter();
-
-        try {
-            if (async)
-                save(tx.isRollbackOnly());
-
-            return tx.isRollbackOnly();
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void commit() {
-        enter();
-
-        try {
-            IgniteInternalFuture<IgniteInternalTx> commitFut = 
cctx.commitTxAsync(tx);
-
-            if (async)
-                saveFuture(commitFut);
-            else
-                commitFut.get();
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() {
-        enter();
-
-        try {
-            cctx.endTx(tx);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void rollback() {
-        enter();
-
-        try {
-            IgniteInternalFuture rollbackFut = cctx.rollbackTxAsync(tx);
-
-            if (async)
-                asyncRes = new IgniteFutureImpl(rollbackFut);
-            else
-                rollbackFut.get();
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * @param res Result to convert to finished future.
-     */
-    private void save(Object res) {
-        asyncRes = new IgniteFinishedFutureImplEx<>(res);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <V1> V1 addMeta(String name, V1 val) {
-        return tx.addMeta(name, val);
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings({"RedundantTypeArguments"})
-    @Override public <V1> V1 meta(String name) {
-        return tx.<V1>meta(name);
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings({"RedundantTypeArguments"})
-    @Override public <V1> V1 removeMeta(String name) {
-        return tx.<V1>removeMeta(name);
-    }
-
-    /**
-     * @param fut Internal future.
-     */
-    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();
-            }
-        });
-
-        asyncRes = new IgniteFutureImpl(fut0);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeObject(tx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, 
ClassNotFoundException {
-        tx = (IgniteInternalTx<K, V>)in.readObject();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(IgniteTxProxyImpl.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionMetricsAdapter.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionMetricsAdapter.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionMetricsAdapter.java
new file mode 100644
index 0000000..2d01693
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionMetricsAdapter.java
@@ -0,0 +1,119 @@
+/*
+ * 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.internal.util.typedef.internal.*;
+import org.apache.ignite.transactions.*;
+
+import java.io.*;
+
+/**
+ * Tx metrics adapter.
+ */
+public class TransactionMetricsAdapter implements TransactionMetrics, 
Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Number of transaction commits. */
+    private volatile int txCommits;
+
+    /** Number of transaction rollbacks. */
+    private volatile int txRollbacks;
+
+    /** Last commit time. */
+    private volatile long commitTime;
+
+    /** Last rollback time. */
+    private volatile long rollbackTime;
+
+    /**
+     *
+     */
+    public TransactionMetricsAdapter() {
+
+    }
+
+    /**
+     * @param m Transaction metrics to copy.
+     */
+    public TransactionMetricsAdapter(TransactionMetrics m) {
+        commitTime = m.commitTime();
+        rollbackTime = m.rollbackTime();
+        txCommits = m.txCommits();
+        txRollbacks = m.txRollbacks();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long commitTime() {
+        return commitTime;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long rollbackTime() {
+        return rollbackTime;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int txCommits() {
+        return txCommits;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int txRollbacks() {
+        return txRollbacks;
+    }
+
+    /**
+     * Transaction commit callback.
+     */
+    public void onTxCommit() {
+        commitTime = U.currentTimeMillis();
+
+        txCommits++;
+    }
+
+    /**
+     * Transaction rollback callback.
+     */
+    public void onTxRollback() {
+        rollbackTime = U.currentTimeMillis();
+
+        txRollbacks++;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeLong(commitTime);
+        out.writeLong(rollbackTime);
+        out.writeInt(txCommits);
+        out.writeInt(txRollbacks);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, 
ClassNotFoundException {
+        commitTime = in.readLong();
+        rollbackTime = in.readLong();
+        txCommits = in.readInt();
+        txRollbacks = in.readInt();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(TransactionMetricsAdapter.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxy.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxy.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxy.java
new file mode 100644
index 0000000..4611081
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxy.java
@@ -0,0 +1,27 @@
+/*
+ * 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.transactions.*;
+
+/**
+ * Marker interface for transaction proxy.
+ */
+public interface TransactionProxy extends Transaction {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java
new file mode 100644
index 0000000..95f54a1
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java
@@ -0,0 +1,345 @@
+/*
+ * 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.internal.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.transactions.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Cache transaction proxy.
+ */
+public class TransactionProxyImpl<K, V> implements TransactionProxy, 
Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Wrapped transaction. */
+    @GridToStringInclude
+    private IgniteInternalTx<K, V> tx;
+
+    /** Gateway. */
+    @GridToStringExclude
+    private GridCacheSharedContext<K, V> cctx;
+
+    /** Async flag. */
+    private boolean async;
+
+    /** Async call result. */
+    private IgniteFuture asyncRes;
+
+    /**
+     * Empty constructor required for {@link Externalizable}.
+     */
+    public TransactionProxyImpl() {
+        // No-op.
+    }
+
+    /**
+     * @param tx Transaction.
+     * @param cctx Shared context.
+     * @param async Async flag.
+     */
+    public TransactionProxyImpl(IgniteInternalTx<K, V> tx, 
GridCacheSharedContext<K, V> cctx, boolean async) {
+        assert tx != null;
+        assert cctx != null;
+
+        this.tx = tx;
+        this.cctx = cctx;
+        this.async = async;
+    }
+
+    /**
+     * @return Transaction.
+     */
+    public IgniteInternalTx<K, V> tx() {
+        return tx;
+    }
+
+    /**
+     * Enters a call.
+     */
+    private void enter() {
+        if (cctx.deploymentEnabled())
+            cctx.deploy().onEnter();
+
+        try {
+            cctx.kernalContext().gateway().readLock();
+        }
+        catch (IllegalStateException e) {
+            throw e;
+        }
+        catch (RuntimeException | Error e) {
+            cctx.kernalContext().gateway().readUnlock();
+
+            throw e;
+        }
+    }
+
+    /**
+     * Leaves a call.
+     */
+    private void leave() {
+        try {
+            CU.unwindEvicts(cctx);
+        }
+        finally {
+            cctx.kernalContext().gateway().readUnlock();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteUuid xid() {
+        return tx.xid();
+    }
+
+    /** {@inheritDoc} */
+    @Override public UUID nodeId() {
+        if (async)
+            save(tx.nodeId());
+
+        return tx.nodeId();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long threadId() {
+        if (async)
+            save(tx.threadId());
+
+        return tx.threadId();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long startTime() {
+        if (async)
+            save(tx.startTime());
+
+        return tx.startTime();
+    }
+
+    /** {@inheritDoc} */
+    @Override public TransactionIsolation isolation() {
+        if (async)
+            save(tx.isolation());
+
+        return tx.isolation();
+    }
+
+    /** {@inheritDoc} */
+    @Override public TransactionConcurrency concurrency() {
+        if (async)
+            save(tx.concurrency());
+
+        return tx.concurrency();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isInvalidate() {
+        if (async)
+            save(tx.isInvalidate());
+
+        return tx.isInvalidate();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean implicit() {
+        if (async)
+            save(tx.implicit());
+
+        return tx.implicit();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long timeout() {
+        if (async)
+            save(tx.timeout());
+
+        return tx.timeout();
+    }
+
+    /** {@inheritDoc} */
+    @Override public TransactionState state() {
+        if (async)
+            save(tx.state());
+
+        return tx.state();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long timeout(long timeout) {
+        return tx.timeout(timeout);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteAsyncSupport withAsync() {
+        return new TransactionProxyImpl<>(tx, cctx, true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isAsync() {
+        return async;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public <R> IgniteFuture<R> future() {
+        return asyncRes;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean setRollbackOnly() {
+        enter();
+
+        try {
+            return tx.setRollbackOnly();
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isRollbackOnly() {
+        enter();
+
+        try {
+            if (async)
+                save(tx.isRollbackOnly());
+
+            return tx.isRollbackOnly();
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void commit() {
+        enter();
+
+        try {
+            IgniteInternalFuture<IgniteInternalTx> commitFut = 
cctx.commitTxAsync(tx);
+
+            if (async)
+                saveFuture(commitFut);
+            else
+                commitFut.get();
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        enter();
+
+        try {
+            cctx.endTx(tx);
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void rollback() {
+        enter();
+
+        try {
+            IgniteInternalFuture rollbackFut = cctx.rollbackTxAsync(tx);
+
+            if (async)
+                asyncRes = new IgniteFutureImpl(rollbackFut);
+            else
+                rollbackFut.get();
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * @param res Result to convert to finished future.
+     */
+    private void save(Object res) {
+        asyncRes = new IgniteFinishedFutureImplEx<>(res);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <V1> V1 addMeta(String name, V1 val) {
+        return tx.addMeta(name, val);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings({"RedundantTypeArguments"})
+    @Override public <V1> V1 meta(String name) {
+        return tx.<V1>meta(name);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings({"RedundantTypeArguments"})
+    @Override public <V1> V1 removeMeta(String name) {
+        return tx.<V1>removeMeta(name);
+    }
+
+    /**
+     * @param fut Internal future.
+     */
+    private void saveFuture(IgniteInternalFuture<IgniteInternalTx> fut) {
+        IgniteInternalFuture<Transaction> fut0 = fut.chain(new 
CX1<IgniteInternalFuture<IgniteInternalTx>, Transaction>() {
+            @Override public Transaction 
applyx(IgniteInternalFuture<IgniteInternalTx> fut) throws 
IgniteCheckedException {
+                return fut.get().proxy();
+            }
+        });
+
+        asyncRes = new IgniteFutureImpl(fut0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(tx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, 
ClassNotFoundException {
+        tx = (IgniteInternalTx<K, V>)in.readObject();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(TransactionProxyImpl.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/GridDataLoadCacheUpdaters.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/GridDataLoadCacheUpdaters.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/GridDataLoadCacheUpdaters.java
index 58c1efd..e2e780b 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/GridDataLoadCacheUpdaters.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/GridDataLoadCacheUpdaters.java
@@ -18,19 +18,12 @@
 package org.apache.ignite.internal.processors.dataload;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.transactions.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
 
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.transactions.IgniteTxConcurrency.*;
-import static org.apache.ignite.transactions.IgniteTxIsolation.*;
-
 /**
  * Bundled factory for cache updaters.
  */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 10e0575..2b55ec5 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -41,8 +41,8 @@ import java.util.concurrent.*;
 import static org.apache.ignite.internal.processors.cache.CacheFlag.*;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.*;
 import static 
org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor.DataStructureType.*;
-import static org.apache.ignite.transactions.IgniteTxConcurrency.*;
-import static org.apache.ignite.transactions.IgniteTxIsolation.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
 
 /**
  * Manager of data structures.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
index 0783d31..f32ff0b 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
@@ -30,8 +30,8 @@ import org.jetbrains.annotations.*;
 import java.io.*;
 import java.util.concurrent.*;
 
-import static org.apache.ignite.transactions.IgniteTxConcurrency.*;
-import static org.apache.ignite.transactions.IgniteTxIsolation.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
 
 /**
  * Cache atomic long implementation.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
index 486be57..8d976fe 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
@@ -30,8 +30,8 @@ import org.jetbrains.annotations.*;
 import java.io.*;
 import java.util.concurrent.*;
 
-import static org.apache.ignite.transactions.IgniteTxConcurrency.*;
-import static org.apache.ignite.transactions.IgniteTxIsolation.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
 
 /**
  * Cache atomic reference implementation.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
index 24df5fc..e6b9c19 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
@@ -34,8 +34,8 @@ import java.util.concurrent.atomic.*;
 import java.util.concurrent.locks.*;
 
 import static java.util.concurrent.TimeUnit.*;
-import static org.apache.ignite.transactions.IgniteTxConcurrency.*;
-import static org.apache.ignite.transactions.IgniteTxIsolation.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
 
 /**
  * Cache sequence implementation.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
index 25f85dc..0717914 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
@@ -32,8 +32,8 @@ import org.jetbrains.annotations.*;
 import java.io.*;
 import java.util.concurrent.*;
 
-import static org.apache.ignite.transactions.IgniteTxConcurrency.*;
-import static org.apache.ignite.transactions.IgniteTxIsolation.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
 
 /**
  * Cache atomic stamped implementation.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
index 187b165..40811b7 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
@@ -31,8 +31,8 @@ import java.io.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
-import static org.apache.ignite.transactions.IgniteTxConcurrency.*;
-import static org.apache.ignite.transactions.IgniteTxIsolation.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
 
 /**
  * Cache count down latch implementation.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridTransactionalCacheQueueImpl.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridTransactionalCacheQueueImpl.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridTransactionalCacheQueueImpl.java
index 0456107..08f15dc 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridTransactionalCacheQueueImpl.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridTransactionalCacheQueueImpl.java
@@ -26,8 +26,8 @@ import org.jetbrains.annotations.*;
 
 import java.util.*;
 
-import static org.apache.ignite.transactions.IgniteTxConcurrency.*;
-import static org.apache.ignite.transactions.IgniteTxIsolation.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
 
 /**
  * {@link org.apache.ignite.IgniteQueue} implementation using transactional 
cache.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
index cf2b5c8..d2f4f23 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
@@ -56,8 +56,8 @@ import static org.apache.ignite.events.EventType.*;
 import static org.apache.ignite.internal.GridTopic.*;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.*;
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.*;
-import static org.apache.ignite.transactions.IgniteTxConcurrency.*;
-import static org.apache.ignite.transactions.IgniteTxIsolation.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
 
 /**
  * Cache based file's data container.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index caa727e..9d1795a 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -42,8 +42,8 @@ import java.util.*;
 import static org.apache.ignite.cache.CacheAtomicityMode.*;
 import static org.apache.ignite.events.EventType.*;
 import static org.apache.ignite.internal.processors.igfs.IgfsFileInfo.*;
-import static org.apache.ignite.transactions.IgniteTxConcurrency.*;
-import static org.apache.ignite.transactions.IgniteTxIsolation.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
 
 /**
  * Cache based structure (meta data) manager.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/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 0fbd2fe..64ab391 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
@@ -44,8 +44,8 @@ import static java.util.concurrent.TimeUnit.*;
 import static org.apache.ignite.internal.GridClosureCallMode.*;
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.*;
 import static 
org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.*;
-import static org.apache.ignite.transactions.IgniteTxConcurrency.*;
-import static org.apache.ignite.transactions.IgniteTxIsolation.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
 
 /**
  * Command handler for API requests.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/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 660fd03..9461b4b 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
@@ -47,8 +47,8 @@ import java.util.concurrent.*;
 import static org.apache.ignite.configuration.DeploymentMode.*;
 import static org.apache.ignite.events.EventType.*;
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.*;
-import static org.apache.ignite.transactions.IgniteTxConcurrency.*;
-import static org.apache.ignite.transactions.IgniteTxIsolation.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
 
 /**
  * Grid service processor.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java 
b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index ac9dc12..103b0ad 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -577,25 +577,25 @@ public abstract class IgniteUtils {
 
         m.put(IgniteTxRollbackCheckedException.class, new 
C1<IgniteCheckedException, IgniteException>() {
             @Override public IgniteException apply(IgniteCheckedException e) {
-                return new IgniteTxRollbackException(e.getMessage(), e);
+                return new TransactionRollbackException(e.getMessage(), e);
             }
         });
 
         m.put(IgniteTxHeuristicCheckedException.class, new 
C1<IgniteCheckedException, IgniteException>() {
             @Override public IgniteException apply(IgniteCheckedException e) {
-                return new IgniteTxHeuristicException(e.getMessage(), e);
+                return new TransactionHeuristicException(e.getMessage(), e);
             }
         });
 
         m.put(IgniteTxTimeoutCheckedException.class, new 
C1<IgniteCheckedException, IgniteException>() {
             @Override public IgniteException apply(IgniteCheckedException e) {
-                return new IgniteTxTimeoutException(e.getMessage(), e);
+                return new TransactionTimeoutException(e.getMessage(), e);
             }
         });
 
         m.put(IgniteTxOptimisticCheckedException.class, new 
C1<IgniteCheckedException, IgniteException>() {
             @Override public IgniteException apply(IgniteCheckedException e) {
-                return new IgniteTxOptimisticException(e.getMessage(), e);
+                return new TransactionOptimisticException(e.getMessage(), e);
             }
         });
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheDefaultConfiguration.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheDefaultConfiguration.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheDefaultConfiguration.java
index 158d79b..ae8aa11 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheDefaultConfiguration.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheDefaultConfiguration.java
@@ -31,10 +31,10 @@ public class VisorCacheDefaultConfiguration implements 
Serializable {
     private static final long serialVersionUID = 0L;
 
     /** Default transaction isolation. */
-    private IgniteTxIsolation txIsolation;
+    private TransactionIsolation txIsolation;
 
     /** Default transaction concurrency. */
-    private IgniteTxConcurrency txConcurrency;
+    private TransactionConcurrency txConcurrency;
 
     /** TTL value. */
     private long ttl;
@@ -70,28 +70,28 @@ public class VisorCacheDefaultConfiguration implements 
Serializable {
     /**
      * @return Default transaction isolation.
      */
-    public IgniteTxIsolation txIsolation() {
+    public TransactionIsolation txIsolation() {
         return txIsolation;
     }
 
     /**
      * @param txIsolation New default transaction isolation.
      */
-    public void txIsolation(IgniteTxIsolation txIsolation) {
+    public void txIsolation(TransactionIsolation txIsolation) {
         this.txIsolation = txIsolation;
     }
 
     /**
      * @return Default transaction concurrency.
      */
-    public IgniteTxConcurrency txConcurrency() {
+    public TransactionConcurrency txConcurrency() {
         return txConcurrency;
     }
 
     /**
      * @param txConcurrency New default transaction concurrency.
      */
-    public void txConcurrency(IgniteTxConcurrency txConcurrency) {
+    public void txConcurrency(TransactionConcurrency txConcurrency) {
         this.txConcurrency = txConcurrency;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorTransactionConfiguration.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorTransactionConfiguration.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorTransactionConfiguration.java
index 7bd2ca8..597336b 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorTransactionConfiguration.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorTransactionConfiguration.java
@@ -31,10 +31,10 @@ public class VisorTransactionConfiguration implements 
Serializable {
     private static final long serialVersionUID = 0L;
 
     /** Default cache concurrency. */
-    private IgniteTxConcurrency dfltConcurrency;
+    private TransactionConcurrency dfltConcurrency;
 
     /** Default transaction isolation. */
-    private IgniteTxIsolation dfltIsolation;
+    private TransactionIsolation dfltIsolation;
 
     /** Default transaction timeout. */
     private long dfltTxTimeout;
@@ -70,28 +70,28 @@ public class VisorTransactionConfiguration implements 
Serializable {
     /**
      * @return Default cache transaction concurrency.
      */
-    public IgniteTxConcurrency defaultTxConcurrency() {
+    public TransactionConcurrency defaultTxConcurrency() {
         return dfltConcurrency;
     }
 
     /**
      * @param dfltConcurrency Default cache transaction concurrency.
      */
-    public void defaultTxConcurrency(IgniteTxConcurrency dfltConcurrency) {
+    public void defaultTxConcurrency(TransactionConcurrency dfltConcurrency) {
         this.dfltConcurrency = dfltConcurrency;
     }
 
     /**
      * @return Default cache transaction isolation.
      */
-    public IgniteTxIsolation defaultTxIsolation() {
+    public TransactionIsolation defaultTxIsolation() {
         return dfltIsolation;
     }
 
     /**
      * @param dfltIsolation Default cache transaction isolation.
      */
-    public void defaultTxIsolation(IgniteTxIsolation dfltIsolation) {
+    public void defaultTxIsolation(TransactionIsolation dfltIsolation) {
         this.dfltIsolation = dfltIsolation;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTx.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTx.java 
b/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTx.java
deleted file mode 100644
index cec5031..0000000
--- a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTx.java
+++ /dev/null
@@ -1,275 +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.transactions;
-
-import org.apache.ignite.*;
-import org.apache.ignite.lang.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-/**
- * Grid cache transaction. Cache transactions have a default 2PC 
(two-phase-commit) behavior and
- * can be plugged into ongoing {@code JTA} transaction by properly implementing
- * {@ignitelink org.apache.ignite.cache.jta.CacheTmLookup}
- * interface. Cache transactions can also be started explicitly directly from 
{@link org.apache.ignite.cache.CacheProjection} API
- * via any of the {@code 'CacheProjection.txStart(..)'} methods.
- * <p>
- * Cache transactions support the following isolation levels:
- * <ul>
- * <li>
- *  {@link IgniteTxIsolation#READ_COMMITTED} isolation level means that always 
a committed value
- *  will be provided for read operations. With this isolation level values are 
always read
- *  from cache global memory or persistent store every time a value is 
accessed. In other words,
- *  if the same key is accessed more than once within the same transaction, it 
may have different
- *  value every time since global cache memory may be updated concurrently by 
other threads.
- * </li>
- * <li>
- *  {@link IgniteTxIsolation#REPEATABLE_READ} isolation level means that if a 
value was read once
- *  within transaction, then all consecutive reads will provide the same 
in-transaction value. With
- *  this isolation level accessed values are stored within in-transaction 
memory, so consecutive access
- *  to the same key within the same transaction will always return the value 
that was previously read or
- *  updated within this transaction. If concurrency is {@link 
IgniteTxConcurrency#PESSIMISTIC}, then a lock
- *  on the key will be acquired prior to accessing the value.
- * </li>
- * <li>
- *  {@link IgniteTxIsolation#SERIALIZABLE} isolation level means that all 
transactions occur in a completely
- *  isolated fashion, as if all transactions in the system had executed 
serially, one after the other.
- *  Read access with this level happens the same way as with {@link 
IgniteTxIsolation#REPEATABLE_READ} level.
- *  However, in {@link IgniteTxConcurrency#OPTIMISTIC} mode, if some 
transactions cannot be serially isolated
- *  from each other, then one winner will be picked and the other transactions 
in conflict will result in
- * {@link 
org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException} 
being thrown.
- * </li>
- * </ul>
- * <p>
- * Cache transactions support the following concurrency models:
- * <ul>
- * <li>
- *  {@link IgniteTxConcurrency#OPTIMISTIC} - in this mode all cache operations 
are not distributed to other
- *  nodes until {@link #commit()} is called. In this mode one {@code 'PREPARE'}
- *  message will be sent to participating cache nodes to start acquiring 
per-transaction locks, and once
- *  all nodes reply {@code 'OK'} (i.e. {@code Phase 1} completes 
successfully), a one-way' {@code 'COMMIT'}
- *  message is sent without waiting for reply. If it is necessary to know 
whenever remote nodes have committed
- *  as well, synchronous commit or synchronous rollback should be enabled via
- *  {@link 
org.apache.ignite.configuration.CacheConfiguration#setWriteSynchronizationMode}
- *  or by setting proper flags on cache projection, such as {@link 
org.apache.ignite.internal.processors.cache.CacheFlag#SYNC_COMMIT}.
- *  <p>
- *  Note that in this mode, optimistic failures are only possible in 
conjunction with
- *  {@link IgniteTxIsolation#SERIALIZABLE} isolation level. In all other 
cases, optimistic
- *  transactions will never fail optimistically and will always be identically 
ordered on all participating
- *  grid nodes.
- * </li>
- * <li>
- *  {@link IgniteTxConcurrency#PESSIMISTIC} - in this mode a lock is acquired 
on all cache operations
- *  with exception of read operations in {@link 
IgniteTxIsolation#READ_COMMITTED} mode. All optional filters
- *  passed into cache operations will be evaluated after successful lock 
acquisition. Whenever
- *  {@link #commit()} is called, a single one-way {@code 'COMMIT'} message
- *  is sent to participating cache nodes without waiting for reply. Note that 
there is no reason for
- *  distributed 'PREPARE' step, as all locks have been already acquired. Just 
like with optimistic mode,
- *  it is possible to configure synchronous commit or rollback and wait till 
transaction commits on
- *  all participating remote nodes.
- * </li>
- * </ul>
- * <p>
- * <h1 class="header">Cache Atomicity Mode</h1>
- * In addition to standard {@link 
org.apache.ignite.cache.CacheAtomicityMode#TRANSACTIONAL} behavior, Ignite also 
supports
- * a lighter {@link org.apache.ignite.cache.CacheAtomicityMode#ATOMIC} mode as 
well. In this mode distributed transactions
- * and distributed locking are not supported. Disabling transactions and 
locking allows to achieve much higher
- * performance and throughput ratios. It is recommended that {@link 
org.apache.ignite.cache.CacheAtomicityMode#ATOMIC} mode
- * is used whenever full {@code ACID}-compliant transactions are not needed.
- * <p>
- * <h1 class="header">Usage</h1>
- * You can use cache transactions as follows:
- * <pre name="code" class="java">
- * Cache&lt;String, Integer&gt; cache = Ignition.ignite().cache();
- *
- * try (GridCacheTx tx = cache.txStart()) {
- *     // Perform transactional operations.
- *     Integer v1 = cache.get("k1");
- *
- *     // Check if v1 satisfies some condition before doing a put.
- *     if (v1 != null && v1 > 0)
- *         cache.putx("k1", 2);
- *
- *     cache.removex("k2");
- *
- *     // Commit the transaction.
- *     tx.commit();
- * }
- * </pre>
- */
-public interface IgniteTx extends AutoCloseable, IgniteAsyncSupport {
-    /**
-     * 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 IgniteException If commit failed.
-     */
-    @IgniteAsyncSupported
-    public void commit() throws IgniteException;
-
-    /**
-     * Ends the transaction. Transaction will be rolled back if it has not 
been committed.
-     *
-     * @throws IgniteException If transaction could not be gracefully ended.
-     */
-    @Override public void close() throws IgniteException;
-
-    /**
-     * Rolls back this transaction.
-     *
-     * @throws IgniteException If rollback failed.
-     */
-    @IgniteAsyncSupported
-    public void rollback() throws IgniteException;
-
-    /**
-     * Removes metadata by name.
-     *
-     * @param name Name of the metadata to remove.
-     * @param <V> Type of the value.
-     * @return Value of removed metadata or {@code null}.
-     */
-    @Nullable public <V> V removeMeta(String name);
-
-    /**
-     * Gets metadata by name.
-     *
-     * @param name Metadata name.
-     * @param <V> Type of the value.
-     * @return Metadata value or {@code null}.
-     */
-    @Nullable public <V> V meta(String name);
-
-    /**
-     * Adds a new metadata.
-     *
-     * @param name Metadata name.
-     * @param val Metadata value.
-     * @param <V> Type of the value.
-     * @return Metadata previously associated with given name, or
-     *      {@code null} if there was none.
-     */
-    @Nullable public <V> V addMeta(String name, V val);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxConcurrency.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxConcurrency.java
 
b/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxConcurrency.java
deleted file mode 100644
index 14213e4..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxConcurrency.java
+++ /dev/null
@@ -1,45 +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.transactions;
-
-import org.jetbrains.annotations.*;
-
-/**
- * Transaction concurrency control. See {@link IgniteTx} for more information
- * on transaction concurrency controls.
- */
-public enum IgniteTxConcurrency {
-    /** Optimistic concurrency control. */
-    OPTIMISTIC,
-
-    /** Pessimistic concurrency control. */
-    PESSIMISTIC;
-
-    /** Enum values. */
-    private static final IgniteTxConcurrency[] VALS = values();
-
-    /**
-     * Efficiently gets enumerated value from its ordinal.
-     *
-     * @param ord Ordinal value.
-     * @return Enumerated value or {@code null} if ordinal out of range.
-     */
-    @Nullable public static IgniteTxConcurrency fromOrdinal(int ord) {
-        return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxHeuristicException.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxHeuristicException.java
 
b/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxHeuristicException.java
deleted file mode 100644
index 55b668b..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxHeuristicException.java
+++ /dev/null
@@ -1,51 +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.transactions;
-
-import org.apache.ignite.*;
-
-/**
- * Exception thrown whenever grid transaction enters an unknown state.
- * This exception is usually thrown whenever commit partially succeeds.
- * Cache will still resolve this situation automatically to ensure data
- * integrity, by invalidating all values participating in this transaction
- * on remote nodes.
- */
-public class IgniteTxHeuristicException extends IgniteException {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     * Creates new heuristic exception with given error message.
-     *
-     * @param msg Error message.
-     */
-    public IgniteTxHeuristicException(String msg) {
-        super(msg);
-    }
-
-    /**
-     * Creates new heuristic exception with given error message and optional 
nested exception.
-     *
-     * @param msg Error message.
-     * @param cause Optional nested exception (can be <tt>null</tt>).
-     */
-    public IgniteTxHeuristicException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxIsolation.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxIsolation.java
 
b/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxIsolation.java
deleted file mode 100644
index 90766c9..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxIsolation.java
+++ /dev/null
@@ -1,49 +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.transactions;
-
-import org.jetbrains.annotations.*;
-
-/**
- * Defines different cache transaction isolation levels. See {@link IgniteTx}
- * documentation for more information about cache transaction isolation levels.
- */
-public enum IgniteTxIsolation {
-    /** Read committed isolation level. */
-    READ_COMMITTED,
-
-    /** Repeatable read isolation level. */
-    REPEATABLE_READ,
-
-    /** Serializable isolation level. */
-    SERIALIZABLE;
-
-    /** Enum values. */
-    private static final IgniteTxIsolation[] VALS = values();
-
-    /**
-     * Efficiently gets enumerated value from its ordinal.
-     *
-     * @param ord Ordinal value.
-     * @return Enumerated value or {@code null} if ordinal out of range.
-     */
-    @Nullable
-    public static IgniteTxIsolation fromOrdinal(int ord) {
-        return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxMetrics.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxMetrics.java
 
b/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxMetrics.java
deleted file mode 100644
index 8ba2b9d..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxMetrics.java
+++ /dev/null
@@ -1,53 +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.transactions;
-
-import java.io.*;
-
-/**
- * Transaction metrics, shared across all caches.
- */
-public interface IgniteTxMetrics extends Serializable {
-    /**
-     * Gets last time transaction was committed.
-     *
-     * @return Last commit time.
-     */
-    public long commitTime();
-
-    /**
-     * Gets last time transaction was rollback.
-     *
-     * @return Last rollback time.
-     */
-    public long rollbackTime();
-
-    /**
-     * Gets total number of transaction commits.
-     *
-     * @return Number of transaction commits.
-     */
-    public int txCommits();
-
-    /**
-     * Gets total number of transaction rollbacks.
-     *
-     * @return Number of transaction rollbacks.
-     */
-    public int txRollbacks();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxOptimisticException.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxOptimisticException.java
 
b/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxOptimisticException.java
deleted file mode 100644
index 95bfa13..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxOptimisticException.java
+++ /dev/null
@@ -1,47 +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.transactions;
-
-import org.apache.ignite.*;
-
-/**
- * Exception thrown whenever grid transactions fail optimistically.
- */
-public class IgniteTxOptimisticException extends IgniteException {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     * Creates new optimistic exception with given error message.
-     *
-     * @param msg Error message.
-     */
-    public IgniteTxOptimisticException(String msg) {
-        super(msg);
-    }
-
-    /**
-     * Creates new optimistic exception with given error message and optional 
nested exception.
-     *
-     * @param msg Error message.
-     * @param cause Optional nested exception (can be <tt>null</tt>).
-     */
-    public IgniteTxOptimisticException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxRollbackException.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxRollbackException.java
 
b/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxRollbackException.java
deleted file mode 100644
index cd5e077..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxRollbackException.java
+++ /dev/null
@@ -1,47 +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.transactions;
-
-import org.apache.ignite.*;
-
-/**
- * Exception thrown whenever grid transactions has been automatically rolled 
back.
- */
-public class IgniteTxRollbackException extends IgniteException {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     * Creates new rollback exception with given error message.
-     *
-     * @param msg Error message.
-     */
-    public IgniteTxRollbackException(String msg) {
-        super(msg);
-    }
-
-    /**
-     * Creates new rollback exception with given error message and optional 
nested exception.
-     *
-     * @param msg Error message.
-     * @param cause Optional nested exception (can be <tt>null</tt>).
-     */
-    public IgniteTxRollbackException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxState.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxState.java 
b/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxState.java
deleted file mode 100644
index 9833ae5..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxState.java
+++ /dev/null
@@ -1,65 +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.transactions;
-
-import org.jetbrains.annotations.*;
-
-/**
- * Cache transaction state.
- */
-public enum IgniteTxState {
-    /** Transaction started. */
-    ACTIVE,
-
-    /** Transaction validating. */
-    PREPARING,
-
-    /** Transaction validation succeeded. */
-    PREPARED,
-
-    /** Transaction is marked for rollback. */
-    MARKED_ROLLBACK,
-
-    /** Transaction commit started (validating finished). */
-    COMMITTING,
-
-    /** Transaction commit succeeded. */
-    COMMITTED,
-
-    /** Transaction rollback started (validation failed). */
-    ROLLING_BACK,
-
-    /** Transaction rollback succeeded. */
-    ROLLED_BACK,
-
-    /** Transaction rollback failed or is otherwise unknown state. */
-    UNKNOWN;
-
-    /** Enumerated values. */
-    private static final IgniteTxState[] VALS = values();
-
-    /**
-     * Efficiently gets enumerated value from its ordinal.
-     *
-     * @param ord Ordinal value.
-     * @return Enumerated value or {@code null} if ordinal out of range.
-     */
-    @Nullable public static IgniteTxState fromOrdinal(int ord) {
-        return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxSynchronization.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxSynchronization.java
 
b/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxSynchronization.java
deleted file mode 100644
index 756e23e..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxSynchronization.java
+++ /dev/null
@@ -1,45 +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.transactions;
-
-import org.jetbrains.annotations.*;
-
-/**
- * Synchronization callback for transaction. You can subscribe to receive 
transaction
- * state change callbacks by registering transaction synchronization via
- * {@link 
org.apache.ignite.cache.GridCache#txSynchronize(IgniteTxSynchronization)} 
method.
- */
-public interface IgniteTxSynchronization {
-    /**
-     * State change callback for transaction. Note that unless transaction has 
been
-     * completed, it is possible to mark it for <tt>rollbackOnly</tt> by 
calling
-     * {@link IgniteTx#setRollbackOnly()} on the passed in transaction.
-     * You can check the return value of {@link IgniteTx#setRollbackOnly() 
setRollbackOnly()}
-     * method to see if transaction was indeed marked for rollback or not.
-     *
-     * @param prevState Previous state of the transaction. If transaction has 
just been
-     *      started, then previous state is {@code null}.
-     * @param newState New state of the transaction. In majority of the cases 
this will be the
-     *      same as {@link IgniteTx#state() tx.state()}, but it is also 
possible
-     *      that transaction may be marked for rollback concurrently with this 
method
-     *      invocation, and in that case <tt>newState</tt> reflects the actual 
state of the
-     *      transition this callback is associated with.
-     * @param tx Transaction whose state has changed.
-     */
-    public void onStateChanged(@Nullable IgniteTxState prevState, 
IgniteTxState newState, IgniteTx tx);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b539033/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxTimeoutException.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxTimeoutException.java
 
b/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxTimeoutException.java
deleted file mode 100644
index 6b6ce9d..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/transactions/IgniteTxTimeoutException.java
+++ /dev/null
@@ -1,47 +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.transactions;
-
-import org.apache.ignite.*;
-
-/**
- * Exception thrown whenever grid transactions time out.
- */
-public class IgniteTxTimeoutException extends IgniteException {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     * Creates new timeout exception with given error message.
-     *
-     * @param msg Error message.
-     */
-    public IgniteTxTimeoutException(String msg) {
-        super(msg);
-    }
-
-    /**
-     * Creates new timeout exception with given error message and optional 
nested exception.
-     *
-     * @param msg Error message.
-     * @param cause Optional nested exception (can be <tt>null</tt>).
-     */
-    public IgniteTxTimeoutException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-}

Reply via email to