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 2017/04/07 09:09:32 UTC

ignite git commit: ignite-4929

Repository: ignite
Updated Branches:
  refs/heads/ignite-4929 [created] b69f43e09


ignite-4929


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

Branch: refs/heads/ignite-4929
Commit: b69f43e09bc9a936ebb863299262950ba308fd09
Parents: a6d518d
Author: sboikov <sb...@gridgain.com>
Authored: Fri Apr 7 11:07:14 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Apr 7 12:02:33 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheUtils.java        |   9 -
 .../distributed/GridDistributedLockRequest.java |  14 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |  24 +--
 .../near/GridNearSingleGetRequest.java          |   4 +-
 .../cache/transactions/IgniteTxAdapter.java     |  13 +-
 .../cache/transactions/IgniteTxEntry.java       |  39 ++--
 .../internal/TestRecordingCommunicationSpi.java |   9 +
 .../cache/IgniteOnePhaseCommitInvokeTest.java   | 207 +++++++++++++++++++
 .../atomic/IgniteCacheAtomicProtocolTest.java   |  16 +-
 .../junits/common/GridCommonAbstractTest.java   |  22 ++
 .../testsuites/IgniteCacheTestSuite2.java       |   7 +-
 11 files changed, 285 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b69f43e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index bce054a..c4e6b6b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -197,15 +197,6 @@ public class GridCacheUtils {
     /** Expire time: must be calculated based on TTL value. */
     public static final long EXPIRE_TIME_CALCULATE = -1L;
 
-    /** Skip store flag bit mask. */
-    public static final int SKIP_STORE_FLAG_MASK = 0x1;
-
-    /** Keep serialized flag. */
-    public static final int KEEP_BINARY_FLAG_MASK = 0x2;
-
-    /** Flag indicating that old value for 'invoke' operation was non null on primary node. */
-    public static final int OLD_VAL_ON_PRIMARY = 0x4;
-
     /** Empty predicate array. */
     private static final IgnitePredicate[] EMPTY = new IgnitePredicate[0];
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b69f43e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
index b1c2c27..74f34a9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
@@ -38,9 +38,6 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.apache.ignite.transactions.TransactionIsolation;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.internal.processors.cache.GridCacheUtils.KEEP_BINARY_FLAG_MASK;
-import static org.apache.ignite.internal.processors.cache.GridCacheUtils.SKIP_STORE_FLAG_MASK;
-
 /**
  * Lock request message.
  */
@@ -48,6 +45,12 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** Skip store flag bit mask. */
+    private static final int SKIP_STORE_FLAG_MASK = 0x01;
+
+    /** Keep binary flag. */
+    private static final int KEEP_BINARY_FLAG_MASK = 0x02;
+
     /** Sender node ID. */
     private UUID nodeId;
 
@@ -90,10 +93,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
     /** Key count. */
     private int txSize;
 
-    /**
-     * Additional flags.
-     * GridCacheUtils.SKIP_STORE_FLAG_MASK - for skipStore flag value.
-     */
+    /** Additional flags. */
     private byte flags;
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b69f43e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index 93ea30d..68e51bb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -64,7 +64,6 @@ import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter;
 import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException;
-import org.apache.ignite.internal.util.F0;
 import org.apache.ignite.internal.util.GridLeanSet;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
@@ -478,7 +477,8 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                 }
 
                 // Send old value in case if rebalancing is not finished.
-                final boolean sndOldVal = !cacheCtx.isLocal() && !cacheCtx.topology().rebalanceFinished(tx.topologyVersion());
+                final boolean sndOldVal = !cacheCtx.isLocal() &&
+                    !cacheCtx.topology().rebalanceFinished(tx.topologyVersion());
 
                 if (sndOldVal) {
                     if (oldVal == null && !readOld) {
@@ -497,11 +497,10 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                             /*keepBinary*/true);
                     }
 
-                    if (oldVal != null) {
+                    if (oldVal != null)
                         oldVal.prepareMarshal(cacheCtx.cacheObjectContext());
 
-                        txEntry.oldValue(oldVal, true);
-                    }
+                    txEntry.oldValue(oldVal);
                 }
             }
             catch (IgniteCheckedException e) {
@@ -1528,21 +1527,6 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
     ) {
         GridDistributedTxMapping global = globalMap.get(n.id());
 
-        if (!F.isEmpty(entry.entryProcessors())) {
-            GridDhtPartitionState state = entry.context().topology().partitionState(n.id(),
-                entry.cached().partition());
-
-            if (state != GridDhtPartitionState.OWNING && state != GridDhtPartitionState.EVICTED) {
-                T2<GridCacheOperation, CacheObject> procVal = entry.entryProcessorCalculatedValue();
-
-                assert procVal != null : entry;
-
-                entry.op(procVal.get1());
-                entry.value(procVal.get2(), true, false);
-                entry.entryProcessors(null);
-            }
-        }
-
         if (global == null)
             globalMap.put(n.id(), global = new GridDistributedTxMapping(n));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b69f43e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java
index 02d2148..5e32c21 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java
@@ -32,8 +32,6 @@ import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.jetbrains.annotations.NotNull;
 
-import static org.apache.ignite.internal.processors.cache.GridCacheUtils.SKIP_STORE_FLAG_MASK;
-
 /**
  *
  */
@@ -205,7 +203,7 @@ public class GridNearSingleGetRequest extends GridCacheMessage implements GridCa
      * @return Read through flag.
      */
     public boolean readThrough() {
-        return (flags & SKIP_STORE_FLAG_MASK) != 0;
+        return (flags & READ_THROUGH_FLAG_MASK) != 0;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b69f43e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index 13ca26a..0bdcafe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -1402,8 +1402,14 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter implement
 
             final boolean keepBinary = txEntry.keepBinary();
 
-            CacheObject cacheVal = txEntry.hasValue() ? txEntry.value() :
-                txEntry.cached().innerGet(
+            CacheObject cacheVal;
+
+            if (txEntry.hasOldValue())
+                cacheVal = txEntry.oldValue();
+            else if (txEntry.hasValue())
+                cacheVal = txEntry.value();
+            else {
+                cacheVal = txEntry.cached().innerGet(
                     null,
                     this,
                     /*swap*/false,
@@ -1412,10 +1418,11 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter implement
                     /*event*/recordEvt,
                     /*temporary*/true,
                     /*subjId*/subjId,
-                    /**closure name */recordEvt ? F.first(txEntry.entryProcessors()).get1() : null,
+                    /*closure name */recordEvt ? F.first(txEntry.entryProcessors()).get1() : null,
                     resolveTaskName(),
                     null,
                     keepBinary);
+            }
 
             boolean modified = false;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b69f43e0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
index 35eab18..f7f61c6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
@@ -37,7 +37,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
 import org.apache.ignite.internal.processors.cache.GridCacheOperation;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
-import org.apache.ignite.internal.processors.cache.GridCacheUtils;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteExternalizableExpiryPolicy;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -57,9 +56,6 @@ import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.READ;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM;
-import static org.apache.ignite.internal.processors.cache.GridCacheUtils.KEEP_BINARY_FLAG_MASK;
-import static org.apache.ignite.internal.processors.cache.GridCacheUtils.OLD_VAL_ON_PRIMARY;
-import static org.apache.ignite.internal.processors.cache.GridCacheUtils.SKIP_STORE_FLAG_MASK;
 
 /**
  * Transaction entry. Note that it is essential that this class does not override
@@ -83,6 +79,15 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
     /** */
     public static final GridCacheVersion GET_ENTRY_INVALID_VER_AFTER_GET = new GridCacheVersion(0, 0, 0, 3);
 
+    /** Skip store flag bit mask. */
+    private static final int TX_ENTRY_SKIP_STORE_FLAG_MASK = 0x01;
+
+    /** Keep binary flag. */
+    private static final int TX_ENTRY_KEEP_BINARY_FLAG_MASK = 0x02;
+
+    /** Flag indicating that old value for 'invoke' operation was non null on primary node. */
+    private static final int TX_ENTRY_OLD_VAL_ON_PRIMARY = 0x04;
+
     /** Prepared flag updater. */
     private static final AtomicIntegerFieldUpdater<IgniteTxEntry> PREPARED_UPD =
         AtomicIntegerFieldUpdater.newUpdater(IgniteTxEntry.class, "prepared");
@@ -194,13 +199,7 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
     /** Expiry policy bytes. */
     private byte[] expiryPlcBytes;
 
-    /**
-     * Additional flags:
-     * <ul>
-     * <li>{@link GridCacheUtils#SKIP_STORE_FLAG_MASK} - for skipStore flag value.</li>
-     * <li>{@link GridCacheUtils#KEEP_BINARY_FLAG_MASK} - for withKeepBinary flag.</li>
-     * </ul>
-     */
+    /** Additional flags. */
     private byte flags;
 
     /** Partition update counter. */
@@ -484,28 +483,28 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
      * @param skipStore Skip store flag.
      */
     public void skipStore(boolean skipStore) {
-        setFlag(skipStore, SKIP_STORE_FLAG_MASK);
+        setFlag(skipStore, TX_ENTRY_SKIP_STORE_FLAG_MASK);
     }
 
     /**
      * @return Skip store flag.
      */
     public boolean skipStore() {
-        return isFlag(SKIP_STORE_FLAG_MASK);
+        return isFlag(TX_ENTRY_SKIP_STORE_FLAG_MASK);
     }
 
     /**
      * @param oldValOnPrimary {@code True} If old value for was non null on primary node.
      */
     public void oldValueOnPrimary(boolean oldValOnPrimary) {
-        setFlag(oldValOnPrimary, OLD_VAL_ON_PRIMARY);
+        setFlag(oldValOnPrimary, TX_ENTRY_OLD_VAL_ON_PRIMARY);
     }
 
     /**
      * @return {@code True} If old value for 'invoke' operation was non null on primary node.
      */
-    public boolean oldValueOnPrimary() {
-        return isFlag(OLD_VAL_ON_PRIMARY);
+    boolean oldValueOnPrimary() {
+        return isFlag(TX_ENTRY_OLD_VAL_ON_PRIMARY);
     }
 
     /**
@@ -514,14 +513,14 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
      * @param keepBinary Keep binary flag value.
      */
     public void keepBinary(boolean keepBinary) {
-        setFlag(keepBinary, KEEP_BINARY_FLAG_MASK);
+        setFlag(keepBinary, TX_ENTRY_KEEP_BINARY_FLAG_MASK);
     }
 
     /**
      * @return Keep binary flag value.
      */
     public boolean keepBinary() {
-        return isFlag(KEEP_BINARY_FLAG_MASK);
+        return isFlag(TX_ENTRY_KEEP_BINARY_FLAG_MASK);
     }
 
     /**
@@ -590,11 +589,11 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
     /**
      * @param oldVal Old value.
      */
-    public void oldValue(CacheObject oldVal, boolean hasOldVal) {
+    public void oldValue(CacheObject oldVal) {
         if (this.oldVal == null)
             this.oldVal = new TxEntryValueHolder();
 
-        this.oldVal.value(op(), oldVal, hasOldVal, hasOldVal);
+        this.oldVal.value(op(), oldVal, true, true);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b69f43e0/modules/core/src/test/java/org/apache/ignite/internal/TestRecordingCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/TestRecordingCommunicationSpi.java b/modules/core/src/test/java/org/apache/ignite/internal/TestRecordingCommunicationSpi.java
index c4d8a79..2fd6f32 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/TestRecordingCommunicationSpi.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/TestRecordingCommunicationSpi.java
@@ -24,6 +24,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.managers.communication.GridIoMessage;
@@ -56,6 +57,14 @@ public class TestRecordingCommunicationSpi extends TcpCommunicationSpi {
     /** */
     private IgnitePredicate<GridIoMessage> blockP;
 
+    /**
+     * @param node Node.
+     * @return Test SPI.
+     */
+    public static TestRecordingCommunicationSpi spi(Ignite node) {
+        return (TestRecordingCommunicationSpi)node.configuration().getCommunicationSpi();
+    }
+
     /** {@inheritDoc} */
     @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure<IgniteException> ackC)
         throws IgniteSpiException {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b69f43e0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteOnePhaseCommitInvokeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteOnePhaseCommitInvokeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteOnePhaseCommitInvokeTest.java
new file mode 100644
index 0000000..adf90e9
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteOnePhaseCommitInvokeTest.java
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import java.util.concurrent.Callable;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cache.CacheEntryProcessor;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.managers.communication.GridIoMessage;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareResponse;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessageV2;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheRebalanceMode.ASYNC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ *
+ */
+public class IgniteOnePhaseCommitInvokeTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        TestRecordingCommunicationSpi commSpi = new TestRecordingCommunicationSpi();
+
+        cfg.setCommunicationSpi(commSpi);
+
+        cfg.setClientMode(client);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setBackups(1);
+        ccfg.setRebalanceMode(ASYNC);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        super.afterTest();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOnePhaseInvoke() throws Exception {
+        boolean flags[] = {true, false};
+
+        for (boolean withOldVal : flags) {
+            for (boolean setVal : flags) {
+                for (boolean retPrev : flags) {
+                    onePhaseInvoke(withOldVal, setVal, retPrev);
+
+                    stopAllGrids();
+                }
+            }
+        }
+    }
+
+    /**
+     * @param withOldVal If {@code true}
+     * @param setVal Flag whether set value from entry processor.
+     * @param retPrev Flag whether entry processor should return previous value.
+     * @throws Exception If failed.
+     */
+    private void onePhaseInvoke(final boolean withOldVal,
+        final boolean setVal,
+        final boolean retPrev)
+        throws Exception
+    {
+        log.info("Test onePhaseInvoke [withOldVal=" + withOldVal + ", setVal=" + setVal + ", retPrev=" + retPrev + ']');
+
+        Ignite srv0 = startGrid(0);
+
+        if (withOldVal)
+            srv0.cache(null).put(1, 1);
+
+        client = true;
+
+        final Ignite clientNode = startGrid(1);
+
+        TestRecordingCommunicationSpi.spi(srv0).blockMessages(new IgnitePredicate<GridIoMessage>() {
+            @Override public boolean apply(GridIoMessage msg0) {
+                Message msg = msg0.message();
+
+                return msg instanceof GridDhtPartitionSupplyMessageV2 &&
+                    ((GridDhtPartitionSupplyMessageV2) msg).cacheId() == CU.cacheId(null);
+            }
+        });
+
+        client = false;
+
+        Ignite srv1 = startGrid(2);
+
+        TestRecordingCommunicationSpi.spi(srv1).blockMessages(GridDhtTxPrepareResponse.class, srv0.name());
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                Object res = clientNode.cache(null).invoke(1, new TestEntryProcessor(setVal, retPrev));
+
+                Object expRes;
+
+                if (retPrev)
+                    expRes = withOldVal ? 1 : null;
+                else
+                    expRes = null;
+
+                assertEquals(expRes, res);
+
+                return null;
+            }
+        });
+
+        U.sleep(1000);
+
+        stopGrid(0);
+
+        fut.get();
+
+        if (!setVal)
+            checkCacheData(F.asMap(1, null), null);
+        else {
+            Object expVal;
+
+            if (setVal)
+                expVal = 2;
+            else
+                expVal = withOldVal ? 1 : null;
+
+            checkCacheData(F.asMap(1, expVal), null);
+        }
+    }
+
+    /**
+     *
+     */
+    static class TestEntryProcessor implements CacheEntryProcessor {
+        /** */
+        private final boolean setVal;
+
+        /** */
+        private final boolean retPrev;
+
+        /**
+         * @param setVal Set value flag.
+         * @param retPrev Return previous value flag.
+         */
+        TestEntryProcessor(boolean setVal, boolean retPrev) {
+            this.setVal = setVal;
+            this.retPrev = retPrev;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object process(MutableEntry e, Object... args) {
+            Object val = e.getValue();
+
+            if (setVal)
+                e.setValue(2);
+
+            return retPrev ? val : null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b69f43e0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/IgniteCacheAtomicProtocolTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/IgniteCacheAtomicProtocolTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/IgniteCacheAtomicProtocolTest.java
index eda030c..b161f89 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/IgniteCacheAtomicProtocolTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/IgniteCacheAtomicProtocolTest.java
@@ -764,21 +764,7 @@ public class IgniteCacheAtomicProtocolTest extends GridCommonAbstractTest {
      * @param expData Expected cache data.
      */
     private void checkData(Map<Integer, Integer> expData) {
-        assert !expData.isEmpty();
-
-        List<Ignite> nodes = G.allGrids();
-
-        assertFalse(nodes.isEmpty());
-
-        for (Ignite node : nodes) {
-            IgniteCache<Integer, Integer> cache = node.cache(TEST_CACHE);
-
-            for (Map.Entry<Integer, Integer> e : expData.entrySet()) {
-                assertEquals("Invalid value [key=" + e.getKey() + ", node=" + node.name() + ']',
-                    e.getValue(),
-                    cache.get(e.getKey()));
-            }
-        }
+        checkCacheData(expData, TEST_CACHE);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b69f43e0/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index 20117bd..78f19a3 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -1295,4 +1295,26 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
             .setEnforceJoinOrder(qry.isEnforceJoinOrder()))
             .getAll().get(0).get(0);
     }
+
+    /**
+     * @param expData Expected cache data.
+     * @param cacheName Cache name.
+     */
+    protected final void checkCacheData(Map<?, ?> expData, String cacheName) {
+        assert !expData.isEmpty();
+
+        List<Ignite> nodes = G.allGrids();
+
+        assertFalse(nodes.isEmpty());
+
+        for (Ignite node : nodes) {
+            IgniteCache<Object, Object> cache = node.cache(cacheName);
+
+            for (Map.Entry<?, ?> e : expData.entrySet()) {
+                assertEquals("Invalid value [key=" + e.getKey() + ", node=" + node.name() + ']',
+                    e.getValue(),
+                    cache.get(e.getKey()));
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b69f43e0/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index e37a8a1..99339b1 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -40,6 +40,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheEntryProcessorNode
 import org.apache.ignite.internal.processors.cache.IgniteCacheIncrementTxTest;
 import org.apache.ignite.internal.processors.cache.IgniteCachePartitionMapUpdateTest;
 import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheAndNodeStop;
+import org.apache.ignite.internal.processors.cache.IgniteOnePhaseCommitInvokeTest;
 import org.apache.ignite.internal.processors.cache.OffheapCacheOnClientsTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLoadingConcurrentGridStartSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLoadingConcurrentGridStartSelfTestAllowOverwrite;
@@ -114,11 +115,11 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePar
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedTxSingleThreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedTxTimeoutSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheRendezvousAffinityClientSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridPartitionedBackupLoadSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheStoreUpdateTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearOffheapCacheStoreUpdateTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridPartitionedBackupLoadSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.NearCacheSyncUpdateTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.NoneRebalanceModeSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridNearOffheapCacheStoreUpdateTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedEvictionSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedJobExecutionTest;
 import org.apache.ignite.internal.processors.cache.local.GridCacheLocalAtomicBasicStoreSelfTest;
@@ -278,6 +279,8 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(GridNearCacheStoreUpdateTest.class));
         suite.addTest(new TestSuite(GridNearOffheapCacheStoreUpdateTest.class));
 
+        suite.addTest(new TestSuite(IgniteOnePhaseCommitInvokeTest.class));
+
         return suite;
     }
 }