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/05/13 11:50:15 UTC

[08/29] incubator-ignite git commit: # ignite-157

# ignite-157


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

Branch: refs/heads/ignite-373
Commit: b3dcbf18407318e7c824a7260fed8c1ec3d3d844
Parents: a2fb8f6
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 8 10:20:30 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 8 11:31:51 2015 +0300

----------------------------------------------------------------------
 .../near/GridAbstractNearTxPrepareFuture.java   | 222 ------------------
 .../near/GridNearOptimisticTxPrepareFuture.java |  15 +-
 .../GridNearPessimisticTxPrepareFuture.java     |  39 +++-
 .../cache/distributed/near/GridNearTxLocal.java |   2 +-
 .../near/GridNearTxPrepareFutureAdapter.java    | 226 +++++++++++++++++++
 .../cache/transactions/IgniteTxHandler.java     |   3 +-
 .../GridCacheAbstractFailoverSelfTest.java      |   7 +
 .../IgniteCacheFailoverTestSuite.java           |   8 +-
 8 files changed, 283 insertions(+), 239 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b3dcbf18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridAbstractNearTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridAbstractNearTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridAbstractNearTxPrepareFuture.java
deleted file mode 100644
index 6f94f21..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridAbstractNearTxPrepareFuture.java
+++ /dev/null
@@ -1,222 +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.distributed.near;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.internal.processors.cache.distributed.*;
-import org.apache.ignite.internal.processors.cache.distributed.dht.*;
-import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.*;
-import org.apache.ignite.internal.processors.cache.transactions.*;
-import org.apache.ignite.internal.processors.cache.version.*;
-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 javax.cache.expiry.*;
-import java.util.*;
-import java.util.concurrent.atomic.*;
-
-import static org.apache.ignite.internal.processors.cache.GridCacheOperation.*;
-
-/**
- * Common code for tx prepare in optimistic and pessimistic modes.
- */
-public abstract class GridAbstractNearTxPrepareFuture extends GridCompoundIdentityFuture<IgniteInternalTx>
-    implements GridCacheFuture<IgniteInternalTx> {
-    /** Logger reference. */
-    protected static final AtomicReference<IgniteLogger> logRef = new AtomicReference<>();
-
-    /** Logger. */
-    protected static IgniteLogger log;
-
-    /** Context. */
-    protected GridCacheSharedContext<?, ?> cctx;
-
-    /** Future ID. */
-    protected IgniteUuid futId;
-
-    /** Transaction. */
-    @GridToStringInclude
-    protected GridNearTxLocal tx;
-
-    /** Error. */
-    @GridToStringExclude
-    protected AtomicReference<Throwable> err = new AtomicReference<>(null);
-
-    /** Trackable flag. */
-    protected boolean trackable = true;
-
-    /** Full information about transaction nodes mapping. */
-    protected GridDhtTxMapping txMapping;
-
-    /**
-     * @param cctx Context.
-     * @param tx Transaction.
-     */
-    public GridAbstractNearTxPrepareFuture(GridCacheSharedContext cctx, final GridNearTxLocal tx) {
-        super(cctx.kernalContext(), new IgniteReducer<IgniteInternalTx, IgniteInternalTx>() {
-            @Override public boolean collect(IgniteInternalTx e) {
-                return true;
-            }
-
-            @Override public IgniteInternalTx reduce() {
-                // Nothing to aggregate.
-                return tx;
-            }
-        });
-
-        assert cctx != null;
-        assert tx != null;
-
-        this.cctx = cctx;
-        this.tx = tx;
-
-        futId = IgniteUuid.randomUuid();
-
-        if (log == null)
-            log = U.logger(cctx.kernalContext(), logRef, GridAbstractNearTxPrepareFuture.class);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteUuid futureId() {
-        return futId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridCacheVersion version() {
-        return tx.xidVersion();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void markNotTrackable() {
-        trackable = false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean trackable() {
-        return trackable;
-    }
-
-    /**
-     * Prepares transaction.
-     */
-    public abstract void prepare();
-
-    /**
-     * @param nodeId Sender.
-     * @param res Result.
-     */
-    public abstract void onResult(UUID nodeId, GridNearTxPrepareResponse res);
-
-    /**
-     * Checks if mapped transaction can be committed on one phase.
-     * One-phase commit can be done if transaction maps to one primary node and not more than one backup.
-     */
-    protected final void checkOnePhase() {
-        if (tx.storeUsed())
-            return;
-
-        Map<UUID, Collection<UUID>> map = txMapping.transactionNodes();
-
-        if (map.size() == 1) {
-            Map.Entry<UUID, Collection<UUID>> entry = F.firstEntry(map);
-
-            assert entry != null;
-
-            Collection<UUID> backups = entry.getValue();
-
-            if (backups.size() <= 1)
-                tx.onePhaseCommit(true);
-        }
-    }
-
-    /**
-     * @param m Mapping.
-     * @param res Response.
-     */
-    protected final void onPrepareResponse(GridDistributedTxMapping m, GridNearTxPrepareResponse res) {
-        if (res == null)
-            return;
-
-        assert res.error() == null : res;
-        assert F.isEmpty(res.invalidPartitions()) : res;
-
-        for (Map.Entry<IgniteTxKey, CacheVersionedValue> entry : res.ownedValues().entrySet()) {
-            IgniteTxEntry txEntry = tx.entry(entry.getKey());
-
-            assert txEntry != null;
-
-            GridCacheContext cacheCtx = txEntry.context();
-
-            while (true) {
-                try {
-                    if (cacheCtx.isNear()) {
-                        GridNearCacheEntry nearEntry = (GridNearCacheEntry)txEntry.cached();
-
-                        CacheVersionedValue tup = entry.getValue();
-
-                        nearEntry.resetFromPrimary(tup.value(), tx.xidVersion(),
-                            tup.version(), m.node().id(), tx.topologyVersion());
-                    }
-                    else if (txEntry.cached().detached()) {
-                        GridDhtDetachedCacheEntry detachedEntry = (GridDhtDetachedCacheEntry)txEntry.cached();
-
-                        CacheVersionedValue tup = entry.getValue();
-
-                        detachedEntry.resetFromPrimary(tup.value(), tx.xidVersion());
-                    }
-
-                    break;
-                }
-                catch (GridCacheEntryRemovedException ignored) {
-                    // Retry.
-                }
-            }
-        }
-
-        tx.implicitSingleResult(res.returnValue());
-
-        for (IgniteTxKey key : res.filterFailedKeys()) {
-            IgniteTxEntry txEntry = tx.entry(key);
-
-            assert txEntry != null : "Missing tx entry for write key: " + key;
-
-            txEntry.op(NOOP);
-
-            assert txEntry.context() != null;
-
-            ExpiryPolicy expiry = txEntry.context().expiryForTxEntry(txEntry);
-
-            if (expiry != null)
-                txEntry.ttl(CU.toTtl(expiry.getExpiryForAccess()));
-        }
-
-        if (!m.empty()) {
-            // Register DHT version.
-            tx.addDhtVersion(m.node().id(), res.dhtVersion());
-
-            m.dhtVersion(res.dhtVersion());
-
-            if (m.near())
-                tx.readyNearLocks(m, res.pending(), res.committedVersions(), res.rolledbackVersions());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b3dcbf18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
index 110cca4..1f2c439 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
@@ -47,7 +47,7 @@ import static org.apache.ignite.transactions.TransactionState.*;
 /**
  *
  */
-public class GridNearOptimisticTxPrepareFuture extends GridAbstractNearTxPrepareFuture
+public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAdapter
     implements GridCacheMvccFuture<IgniteInternalTx> {
     /** */
     private Collection<IgniteTxKey> lockKeys = new GridConcurrentHashSet<>();
@@ -397,8 +397,7 @@ public class GridNearOptimisticTxPrepareFuture extends GridAbstractNearTxPrepare
 
         txMapping = new GridDhtTxMapping();
 
-        ConcurrentLinkedDeque8<GridDistributedTxMapping> mappings =
-            new ConcurrentLinkedDeque8<>();
+        ConcurrentLinkedDeque8<GridDistributedTxMapping> mappings = new ConcurrentLinkedDeque8<>();
 
         if (!F.isEmpty(reads) || !F.isEmpty(writes)) {
             for (int cacheId : tx.activeCacheIds()) {
@@ -647,7 +646,15 @@ public class GridNearOptimisticTxPrepareFuture extends GridAbstractNearTxPrepare
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return S.toString(GridNearOptimisticTxPrepareFuture.class, this, super.toString());
+        Collection<String> pendingFuts = F.viewReadOnly(pending(), new C1<IgniteInternalFuture<?>, String>() {
+            @Override public String apply(IgniteInternalFuture<?> f) {
+                return "[node=" + ((MiniFuture)f).node().id() + ", loc=" + ((MiniFuture)f).node().isLocal() + "]";
+            }
+        });
+
+        return S.toString(GridNearOptimisticTxPrepareFuture.class, this,
+            "pendingFuts", pendingFuts,
+            "super", super.toString());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b3dcbf18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
index e3f24f5..ba8b92c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
@@ -41,7 +41,7 @@ import static org.apache.ignite.transactions.TransactionState.*;
 /**
  *
  */
-public class GridNearPessimisticTxPrepareFuture extends GridAbstractNearTxPrepareFuture {
+public class GridNearPessimisticTxPrepareFuture extends GridNearTxPrepareFutureAdapter {
     /**
      * @param cctx Context.
      * @param tx Transaction.
@@ -72,7 +72,7 @@ public class GridNearPessimisticTxPrepareFuture extends GridAbstractNearTxPrepar
             MiniFuture f = (MiniFuture)fut;
 
             if (f.node().id().equals(nodeId)) {
-                f.onError(new ClusterTopologyCheckedException("Remote node left grid: " + nodeId));
+                f.onNodeLeft(new ClusterTopologyCheckedException("Remote node left grid: " + nodeId));
 
                 found = true;
             }
@@ -222,8 +222,10 @@ public class GridNearPessimisticTxPrepareFuture extends GridAbstractNearTxPrepar
                 try {
                     cctx.io().send(node, req, tx.ioPolicy());
                 }
+                catch (ClusterTopologyCheckedException e) {
+                    fut.onNodeLeft(e);
+                }
                 catch (IgniteCheckedException e) {
-                    // Fail the whole thing.
                     fut.onError(e);
                 }
             }
@@ -242,7 +244,7 @@ public class GridNearPessimisticTxPrepareFuture extends GridAbstractNearTxPrepar
         if (err == null)
             tx.state(PREPARED);
 
-        if (super.onDone(res, err)) {
+        if (super.onDone(tx, err)) {
             cctx.mvcc().removeFuture(this);
 
             return true;
@@ -253,7 +255,15 @@ public class GridNearPessimisticTxPrepareFuture extends GridAbstractNearTxPrepar
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return S.toString(GridNearPessimisticTxPrepareFuture.class, this, super.toString());
+        Collection<String> pendingFuts = F.viewReadOnly(pending(), new C1<IgniteInternalFuture<?>, String>() {
+            @Override public String apply(IgniteInternalFuture<?> f) {
+                return "[node=" + ((MiniFuture)f).node().id() + ", loc=" + ((MiniFuture)f).node().isLocal() + "]";
+            }
+        });
+
+        return S.toString(GridNearPessimisticTxPrepareFuture.class, this,
+            "pendingFuts", pendingFuts,
+            "super", super.toString());
     }
 
     /**
@@ -306,8 +316,25 @@ public class GridNearPessimisticTxPrepareFuture extends GridAbstractNearTxPrepar
         /**
          * @param e Error.
          */
+        void onNodeLeft(ClusterTopologyCheckedException e) {
+            onError(e);
+        }
+
+        /**
+         * @param e Error.
+         */
         void onError(Throwable e) {
-            err.compareAndSet(null, e);
+            if (isDone()) {
+                U.warn(log, "Received error when future is done [fut=" + this + ", err=" + e + ", tx=" + tx + ']');
+
+                return;
+            }
+
+            if (log.isDebugEnabled())
+                log.debug("Error on tx prepare [fut=" + this + ", err=" + e + ", tx=" + tx +  ']');
+
+            if (err.compareAndSet(null, e))
+                tx.setRollbackOnly();
 
             onDone(e);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b3dcbf18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index a003d19..50d3f3e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -682,7 +682,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> prepareAsync() {
-        GridAbstractNearTxPrepareFuture fut = (GridAbstractNearTxPrepareFuture)prepFut.get();
+        GridNearTxPrepareFutureAdapter fut = (GridNearTxPrepareFutureAdapter)prepFut.get();
 
         if (fut == null) {
             // Future must be created before any exception can be thrown.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b3dcbf18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFutureAdapter.java
new file mode 100644
index 0000000..60b918c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFutureAdapter.java
@@ -0,0 +1,226 @@
+/*
+ * 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.distributed.near;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.cache.distributed.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.*;
+import org.apache.ignite.internal.processors.cache.transactions.*;
+import org.apache.ignite.internal.processors.cache.version.*;
+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 javax.cache.expiry.*;
+import java.util.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.internal.processors.cache.GridCacheOperation.*;
+
+/**
+ * Common code for tx prepare in optimistic and pessimistic modes.
+ */
+public abstract class GridNearTxPrepareFutureAdapter extends GridCompoundIdentityFuture<IgniteInternalTx>
+    implements GridCacheFuture<IgniteInternalTx> {
+    /** Logger reference. */
+    protected static final AtomicReference<IgniteLogger> logRef = new AtomicReference<>();
+
+    /** */
+    private static final IgniteReducer<IgniteInternalTx, IgniteInternalTx> REDUCER =
+        new IgniteReducer<IgniteInternalTx, IgniteInternalTx>() {
+            @Override public boolean collect(IgniteInternalTx e) {
+                return true;
+            }
+
+            @Override public IgniteInternalTx reduce() {
+                // Nothing to aggregate.
+                return null;
+            }
+        };
+
+    /** Logger. */
+    protected static IgniteLogger log;
+
+    /** Context. */
+    protected GridCacheSharedContext<?, ?> cctx;
+
+    /** Future ID. */
+    protected IgniteUuid futId;
+
+    /** Transaction. */
+    @GridToStringInclude
+    protected GridNearTxLocal tx;
+
+    /** Error. */
+    @GridToStringExclude
+    protected AtomicReference<Throwable> err = new AtomicReference<>(null);
+
+    /** Trackable flag. */
+    protected boolean trackable = true;
+
+    /** Full information about transaction nodes mapping. */
+    protected GridDhtTxMapping txMapping;
+
+    /**
+     * @param cctx Context.
+     * @param tx Transaction.
+     */
+    public GridNearTxPrepareFutureAdapter(GridCacheSharedContext cctx, final GridNearTxLocal tx) {
+        super(cctx.kernalContext(), REDUCER);
+
+        assert cctx != null;
+        assert tx != null;
+
+        this.cctx = cctx;
+        this.tx = tx;
+
+        futId = IgniteUuid.randomUuid();
+
+        if (log == null)
+            log = U.logger(cctx.kernalContext(), logRef, GridNearTxPrepareFutureAdapter.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteUuid futureId() {
+        return futId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridCacheVersion version() {
+        return tx.xidVersion();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void markNotTrackable() {
+        trackable = false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean trackable() {
+        return trackable;
+    }
+
+    /**
+     * Prepares transaction.
+     */
+    public abstract void prepare();
+
+    /**
+     * @param nodeId Sender.
+     * @param res Result.
+     */
+    public abstract void onResult(UUID nodeId, GridNearTxPrepareResponse res);
+
+    /**
+     * Checks if mapped transaction can be committed on one phase.
+     * One-phase commit can be done if transaction maps to one primary node and not more than one backup.
+     */
+    protected final void checkOnePhase() {
+        if (tx.storeUsed())
+            return;
+
+        Map<UUID, Collection<UUID>> map = txMapping.transactionNodes();
+
+        if (map.size() == 1) {
+            Map.Entry<UUID, Collection<UUID>> entry = F.firstEntry(map);
+
+            assert entry != null;
+
+            Collection<UUID> backups = entry.getValue();
+
+            if (backups.size() <= 1)
+                tx.onePhaseCommit(true);
+        }
+    }
+
+    /**
+     * @param m Mapping.
+     * @param res Response.
+     */
+    protected final void onPrepareResponse(GridDistributedTxMapping m, GridNearTxPrepareResponse res) {
+        if (res == null)
+            return;
+
+        assert res.error() == null : res;
+        assert F.isEmpty(res.invalidPartitions()) : res;
+
+        for (Map.Entry<IgniteTxKey, CacheVersionedValue> entry : res.ownedValues().entrySet()) {
+            IgniteTxEntry txEntry = tx.entry(entry.getKey());
+
+            assert txEntry != null;
+
+            GridCacheContext cacheCtx = txEntry.context();
+
+            while (true) {
+                try {
+                    if (cacheCtx.isNear()) {
+                        GridNearCacheEntry nearEntry = (GridNearCacheEntry)txEntry.cached();
+
+                        CacheVersionedValue tup = entry.getValue();
+
+                        nearEntry.resetFromPrimary(tup.value(), tx.xidVersion(),
+                            tup.version(), m.node().id(), tx.topologyVersion());
+                    }
+                    else if (txEntry.cached().detached()) {
+                        GridDhtDetachedCacheEntry detachedEntry = (GridDhtDetachedCacheEntry)txEntry.cached();
+
+                        CacheVersionedValue tup = entry.getValue();
+
+                        detachedEntry.resetFromPrimary(tup.value(), tx.xidVersion());
+                    }
+
+                    break;
+                }
+                catch (GridCacheEntryRemovedException ignored) {
+                    // Retry.
+                }
+            }
+        }
+
+        tx.implicitSingleResult(res.returnValue());
+
+        for (IgniteTxKey key : res.filterFailedKeys()) {
+            IgniteTxEntry txEntry = tx.entry(key);
+
+            assert txEntry != null : "Missing tx entry for write key: " + key;
+
+            txEntry.op(NOOP);
+
+            assert txEntry.context() != null;
+
+            ExpiryPolicy expiry = txEntry.context().expiryForTxEntry(txEntry);
+
+            if (expiry != null)
+                txEntry.ttl(CU.toTtl(expiry.getExpiryForAccess()));
+        }
+
+        if (!m.empty()) {
+            // Register DHT version.
+            tx.addDhtVersion(m.node().id(), res.dhtVersion());
+
+            m.dhtVersion(res.dhtVersion());
+
+            if (m.near())
+                tx.readyNearLocks(m, res.pending(), res.committedVersions(), res.rolledbackVersions());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b3dcbf18/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 a403f28..826f392 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
@@ -31,7 +31,6 @@ import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -350,7 +349,7 @@ public class IgniteTxHandler {
      * @param res Response.
      */
     private void processNearTxPrepareResponse(UUID nodeId, GridNearTxPrepareResponse res) {
-        GridAbstractNearTxPrepareFuture fut = (GridAbstractNearTxPrepareFuture)ctx.mvcc()
+        GridNearTxPrepareFutureAdapter fut = (GridNearTxPrepareFutureAdapter)ctx.mvcc()
             .<IgniteInternalTx>future(res.version(), res.futureId());
 
         if (fut == null) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b3dcbf18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
index 5389ef9..6f6355a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.transactions.*;
 import org.jetbrains.annotations.*;
@@ -70,6 +71,12 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
 
         cfg.getTransactionConfiguration().setTxSerializableEnabled(true);
 
+        TcpDiscoverySpi discoSpi = (TcpDiscoverySpi)cfg.getDiscoverySpi();
+
+        discoSpi.setSocketTimeout(10_000);
+        discoSpi.setAckTimeout(10_000);
+        discoSpi.setNetworkTimeout(10_000);
+
         return cfg;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b3dcbf18/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
index 2acd6a3..2cc6a5a 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
@@ -54,10 +54,6 @@ public class IgniteCacheFailoverTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheAtomicPrimaryWriteOrderFailoverSelfTest.class);
         suite.addTestSuite(GridCacheAtomicReplicatedFailoverSelfTest.class);
 
-        suite.addTestSuite(GridCachePartitionedFailoverSelfTest.class);
-        suite.addTestSuite(GridCacheColocatedFailoverSelfTest.class);
-        suite.addTestSuite(GridCacheReplicatedFailoverSelfTest.class);
-
         suite.addTestSuite(IgniteCacheAtomicNodeJoinTest.class);
         suite.addTestSuite(IgniteCacheTxNodeJoinTest.class);
         suite.addTestSuite(IgniteCacheTxFairAffinityNodeJoinTest.class);
@@ -65,6 +61,10 @@ public class IgniteCacheFailoverTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheTxNearDisabledPutGetRestartTest.class);
         suite.addTestSuite(IgniteCacheTxNearDisabledFairAffinityPutGetRestartTest.class);
 
+        suite.addTestSuite(GridCachePartitionedFailoverSelfTest.class);
+        suite.addTestSuite(GridCacheColocatedFailoverSelfTest.class);
+        suite.addTestSuite(GridCacheReplicatedFailoverSelfTest.class);
+
         return suite;
     }
 }