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/20 14:41:18 UTC

[09/28] incubator-ignite git commit: #ignite-797: Remove 'groupLock' logic from cache code.

#ignite-797: Remove 'groupLock' logic from cache code.


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

Branch: refs/heads/ignite-456
Commit: da5a2282edc8e32d58d504d4ce96d5e88ab046e5
Parents: 94e202a
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri May 15 18:04:23 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri May 15 18:04:23 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      |   8 +-
 .../processors/cache/GridCacheMapEntry.java     |  35 +---
 .../distributed/GridDistributedLockRequest.java | 111 +++----------
 .../GridDistributedTxFinishRequest.java         |  70 ++------
 .../GridDistributedTxPrepareRequest.java        | 112 +++----------
 .../GridDistributedTxRemoteAdapter.java         |  20 +--
 .../distributed/dht/GridDhtLockFuture.java      |   2 -
 .../distributed/dht/GridDhtLockRequest.java     |  45 +++--
 .../dht/GridDhtTransactionalCacheAdapter.java   |   6 -
 .../distributed/dht/GridDhtTxFinishFuture.java  |   3 -
 .../distributed/dht/GridDhtTxFinishRequest.java |  43 +++--
 .../cache/distributed/dht/GridDhtTxLocal.java   |   6 -
 .../distributed/dht/GridDhtTxLocalAdapter.java  |  68 +-------
 .../distributed/dht/GridDhtTxPrepareFuture.java |  18 +-
 .../dht/GridDhtTxPrepareRequest.java            |  60 ++++---
 .../cache/distributed/dht/GridDhtTxRemote.java  |   8 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   6 -
 .../distributed/near/GridNearLockFuture.java    |   6 -
 .../distributed/near/GridNearLockRequest.java   |  61 +++----
 .../near/GridNearOptimisticTxPrepareFuture.java |  15 +-
 .../GridNearPessimisticTxPrepareFuture.java     |   2 -
 .../near/GridNearTransactionalCache.java        |   4 -
 .../near/GridNearTxFinishRequest.java           |  28 ++--
 .../cache/distributed/near/GridNearTxLocal.java |  20 +--
 .../near/GridNearTxPrepareRequest.java          |  52 +++---
 .../distributed/near/GridNearTxRemote.java      |  24 +--
 .../cache/transactions/IgniteInternalTx.java    |  10 --
 .../transactions/IgniteTransactionsImpl.java    |   4 +-
 .../cache/transactions/IgniteTxAdapter.java     |  72 +-------
 .../cache/transactions/IgniteTxEntry.java       |  48 +-----
 .../cache/transactions/IgniteTxHandler.java     |   6 -
 .../transactions/IgniteTxLocalAdapter.java      | 165 ++-----------------
 .../cache/transactions/IgniteTxLocalEx.java     |  21 +--
 .../cache/transactions/IgniteTxManager.java     |  62 +------
 .../processors/cache/jta/CacheJtaManager.java   |   4 +-
 35 files changed, 239 insertions(+), 986 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 4106cb0..8d7b135 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -3906,9 +3906,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 READ_COMMITTED,
                 tCfg.getDefaultTxTimeout(),
                 !ctx.skipStore(),
-                0,
-                /** group lock keys */null,
-                /** partition lock */false
+                0
             );
 
             assert tx != null;
@@ -3977,9 +3975,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 READ_COMMITTED,
                 ctx.kernalContext().config().getTransactionConfiguration().getDefaultTxTimeout(),
                 !ctx.skipStore(),
-                0,
-                null,
-                false);
+                0);
 
         return asyncOp(tx, op);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 86ed57a..92035af 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -961,13 +961,8 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
         synchronized (this) {
             checkObsolete();
 
-            if (cctx.kernalContext().config().isCacheSanityCheckEnabled()) {
-                if (tx != null && tx.groupLock())
-                    groupLockSanityCheck(tx);
-                else
-                    assert tx == null || (!tx.local() && tx.onePhaseCommit()) || tx.ownsLock(this) :
-                        "Transaction does not own lock for update [entry=" + this + ", tx=" + tx + ']';
-            }
+            assert tx == null || (!tx.local() && tx.onePhaseCommit()) || tx.ownsLock(this) :
+                "Transaction does not own lock for update [entry=" + this + ", tx=" + tx + ']';
 
             // Load and remove from swap if it is new.
             boolean startVer = isStartVersion();
@@ -1125,10 +1120,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
         synchronized (this) {
             checkObsolete();
 
-            if (tx != null && tx.groupLock() && cctx.kernalContext().config().isCacheSanityCheckEnabled())
-                groupLockSanityCheck(tx);
-            else
-                assert tx == null || (!tx.local() && tx.onePhaseCommit()) || tx.ownsLock(this) :
+            assert tx == null || (!tx.local() && tx.onePhaseCommit()) || tx.ownsLock(this) :
                     "Transaction does not own lock for remove[entry=" + this + ", tx=" + tx + ']';
 
             boolean startVer = isStartVersion();
@@ -1195,7 +1187,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
                 obsoleteVer = newVer;
             else {
                 // Only delete entry if the lock is not explicit.
-                if (tx.groupLock() || lockedBy(tx.xidVersion()))
+                if (lockedBy(tx.xidVersion()))
                     obsoleteVer = tx.xidVersion();
                 else if (log.isDebugEnabled())
                     log.debug("Obsolete version was not set because lock was explicit: " + this);
@@ -2796,25 +2788,6 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
     }
 
     /**
-     * Checks that entries in group locks transactions are not locked during commit.
-     *
-     * @param tx Transaction to check.
-     * @throws GridCacheEntryRemovedException If entry is obsolete.
-     * @throws IgniteCheckedException If entry was externally locked.
-     */
-    private void groupLockSanityCheck(IgniteInternalTx tx) throws GridCacheEntryRemovedException, IgniteCheckedException {
-        assert tx.groupLock();
-
-        IgniteTxEntry txEntry = tx.entry(txKey());
-
-        if (txEntry.groupLockEntry()) {
-            if (lockedByAny())
-                throw new IgniteCheckedException("Failed to update cache entry (entry was externally locked while " +
-                    "accessing entry within group lock transaction) [entry=" + this + ", tx=" + tx + ']');
-        }
-    }
-
-    /**
      * @param failFast Fail fast flag.
      * @param topVer Topology version.
      * @param filter Filter.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/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 fd1040f..c5ac847 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
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed;
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -83,12 +82,6 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
     /** Key count. */
     private int txSize;
 
-    /** Group lock key if this is a group-lock transaction. */
-    private IgniteTxKey grpLockKey;
-
-    /** Partition lock flag. Only if group-lock transaction. */
-    private boolean partLock;
-
     /**
      * Additional flags.
      * GridCacheUtils.SKIP_STORE_FLAG_MASK - for skipStore flag value.
@@ -116,9 +109,6 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
      * @param timeout Lock timeout.
      * @param keyCnt Number of keys.
      * @param txSize Expected transaction size.
-     * @param grpLockKey Group lock key if this is a group-lock transaction.
-     * @param partLock {@code True} if this is a group-lock transaction request and whole partition is
-     *      locked.
      * @param skipStore Skip store flag.
      */
     public GridDistributedLockRequest(
@@ -135,8 +125,6 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
         long timeout,
         int keyCnt,
         int txSize,
-        @Nullable IgniteTxKey grpLockKey,
-        boolean partLock,
         boolean skipStore
     ) {
         super(lockVer, keyCnt);
@@ -156,8 +144,6 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
         this.isInvalidate = isInvalidate;
         this.timeout = timeout;
         this.txSize = txSize;
-        this.grpLockKey = grpLockKey;
-        this.partLock = partLock;
 
         retVals = new boolean[keyCnt];
 
@@ -295,27 +281,6 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
     }
 
     /**
-     * @return {@code True} if lock request for group-lock transaction.
-     */
-    public boolean groupLock() {
-        return grpLockKey != null;
-    }
-
-    /**
-     * @return Group lock key.
-     */
-    @Nullable public IgniteTxKey groupLockKey() {
-        return grpLockKey;
-    }
-
-    /**
-     * @return {@code True} if partition is locked in group-lock transaction.
-     */
-    public boolean partitionLock() {
-        return partLock;
-    }
-
-    /**
      * @return Max lock wait time.
      */
     public long timeout() {
@@ -330,9 +295,6 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
         GridCacheContext cctx = ctx.cacheContext(cacheId);
 
         prepareMarshalCacheObjects(keys, cctx);
-
-        if (grpLockKey != null)
-            grpLockKey.prepareMarshal(cctx);
     }
 
     /** {@inheritDoc} */
@@ -342,9 +304,6 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
         GridCacheContext cctx = ctx.cacheContext(cacheId);
 
         finishUnmarshalCacheObjects(keys, cctx, ldr);
-
-        if (grpLockKey != null)
-            grpLockKey.finishUnmarshal(cctx, ldr);
     }
 
     /** {@inheritDoc} */
@@ -375,78 +334,66 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
                 writer.incrementState();
 
             case 10:
-                if (!writer.writeMessage("grpLockKey", grpLockKey))
-                    return false;
-
-                writer.incrementState();
-
-            case 11:
                 if (!writer.writeBoolean("isInTx", isInTx))
                     return false;
 
                 writer.incrementState();
 
-            case 12:
+            case 11:
                 if (!writer.writeBoolean("isInvalidate", isInvalidate))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
+            case 12:
                 if (!writer.writeBoolean("isRead", isRead))
                     return false;
 
                 writer.incrementState();
 
-            case 14:
+            case 13:
                 if (!writer.writeByte("isolation", isolation != null ? (byte)isolation.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
 
-            case 15:
+            case 14:
                 if (!writer.writeCollection("keys", keys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 16:
+            case 15:
                 if (!writer.writeMessage("nearXidVer", nearXidVer))
                     return false;
 
                 writer.incrementState();
 
-            case 17:
+            case 16:
                 if (!writer.writeUuid("nodeId", nodeId))
                     return false;
 
                 writer.incrementState();
 
-            case 18:
-                if (!writer.writeBoolean("partLock", partLock))
-                    return false;
-
-                writer.incrementState();
-
-            case 19:
+            case 17:
                 if (!writer.writeBooleanArray("retVals", retVals))
                     return false;
 
                 writer.incrementState();
 
-            case 20:
+            case 18:
                 if (!writer.writeLong("threadId", threadId))
                     return false;
 
                 writer.incrementState();
 
-            case 21:
+            case 19:
                 if (!writer.writeLong("timeout", timeout))
                     return false;
 
                 writer.incrementState();
 
-            case 22:
+            case 20:
                 if (!writer.writeInt("txSize", txSize))
                     return false;
 
@@ -485,14 +432,6 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
                 reader.incrementState();
 
             case 10:
-                grpLockKey = reader.readMessage("grpLockKey");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 11:
                 isInTx = reader.readBoolean("isInTx");
 
                 if (!reader.isLastRead())
@@ -500,7 +439,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 12:
+            case 11:
                 isInvalidate = reader.readBoolean("isInvalidate");
 
                 if (!reader.isLastRead())
@@ -508,7 +447,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 13:
+            case 12:
                 isRead = reader.readBoolean("isRead");
 
                 if (!reader.isLastRead())
@@ -516,7 +455,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 14:
+            case 13:
                 byte isolationOrd;
 
                 isolationOrd = reader.readByte("isolation");
@@ -528,7 +467,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 15:
+            case 14:
                 keys = reader.readCollection("keys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -536,7 +475,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 16:
+            case 15:
                 nearXidVer = reader.readMessage("nearXidVer");
 
                 if (!reader.isLastRead())
@@ -544,7 +483,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 17:
+            case 16:
                 nodeId = reader.readUuid("nodeId");
 
                 if (!reader.isLastRead())
@@ -552,15 +491,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 18:
-                partLock = reader.readBoolean("partLock");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 19:
+            case 17:
                 retVals = reader.readBooleanArray("retVals");
 
                 if (!reader.isLastRead())
@@ -568,7 +499,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 20:
+            case 18:
                 threadId = reader.readLong("threadId");
 
                 if (!reader.isLastRead())
@@ -576,7 +507,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 21:
+            case 19:
                 timeout = reader.readLong("timeout");
 
                 if (!reader.isLastRead())
@@ -584,7 +515,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 22:
+            case 20:
                 txSize = reader.readInt("txSize");
 
                 if (!reader.isLastRead())
@@ -604,7 +535,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 23;
+        return 21;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishRequest.java
index 9672a75..c524575 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishRequest.java
@@ -18,10 +18,8 @@
 package org.apache.ignite.internal.processors.cache.distributed;
 
 import org.apache.ignite.*;
-import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.lang.*;
@@ -66,9 +64,6 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage {
     /** Expected txSize. */
     private int txSize;
 
-    /** Group lock key. */
-    private IgniteTxKey grpLockKey;
-
     /** System transaction flag. */
     private boolean sys;
 
@@ -95,7 +90,6 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage {
      * @param committedVers Committed versions.
      * @param rolledbackVers Rolled back versions.
      * @param txSize Expected transaction size.
-     * @param grpLockKey Group lock key if this is a group-lock transaction.
      */
     public GridDistributedTxFinishRequest(
         GridCacheVersion xidVer,
@@ -111,8 +105,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage {
         GridCacheVersion baseVer,
         Collection<GridCacheVersion> committedVers,
         Collection<GridCacheVersion> rolledbackVers,
-        int txSize,
-        @Nullable IgniteTxKey grpLockKey
+        int txSize
     ) {
         super(xidVer, 0);
         assert xidVer != null;
@@ -128,7 +121,6 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage {
         this.syncRollback = syncRollback;
         this.baseVer = baseVer;
         this.txSize = txSize;
-        this.grpLockKey = grpLockKey;
 
         completedVersions(committedVers, rolledbackVers);
     }
@@ -219,35 +211,15 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage {
         return commit ? syncCommit : syncRollback;
     }
 
-    /**
-     * @return {@code True} if group lock transaction.
-     */
-    public boolean groupLock() {
-        return grpLockKey != null;
-    }
-
-    /**
-     * @return Group lock key.
-     */
-    @Nullable public IgniteTxKey groupLockKey() {
-        return grpLockKey;
-    }
-
     /** {@inheritDoc}
      * @param ctx*/
     @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException {
         super.prepareMarshal(ctx);
-
-        if (grpLockKey != null)
-            grpLockKey.prepareMarshal(ctx.cacheContext(cacheId));
     }
 
     /** {@inheritDoc} */
     @Override public void finishUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException {
         super.finishUnmarshal(ctx, ldr);
-
-        if (grpLockKey != null)
-            grpLockKey.finishUnmarshal(ctx.cacheContext(cacheId), ldr);
     }
 
     /** {@inheritDoc} */
@@ -290,48 +262,42 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage {
                 writer.incrementState();
 
             case 12:
-                if (!writer.writeMessage("grpLockKey", grpLockKey))
-                    return false;
-
-                writer.incrementState();
-
-            case 13:
                 if (!writer.writeBoolean("invalidate", invalidate))
                     return false;
 
                 writer.incrementState();
 
-            case 14:
+            case 13:
                 if (!writer.writeByte("plc", plc != null ? (byte)plc.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
 
-            case 15:
+            case 14:
                 if (!writer.writeBoolean("syncCommit", syncCommit))
                     return false;
 
                 writer.incrementState();
 
-            case 16:
+            case 15:
                 if (!writer.writeBoolean("syncRollback", syncRollback))
                     return false;
 
                 writer.incrementState();
 
-            case 17:
+            case 16:
                 if (!writer.writeBoolean("sys", sys))
                     return false;
 
                 writer.incrementState();
 
-            case 18:
+            case 17:
                 if (!writer.writeLong("threadId", threadId))
                     return false;
 
                 writer.incrementState();
 
-            case 19:
+            case 18:
                 if (!writer.writeInt("txSize", txSize))
                     return false;
 
@@ -386,14 +352,6 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage {
                 reader.incrementState();
 
             case 12:
-                grpLockKey = reader.readMessage("grpLockKey");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 13:
                 invalidate = reader.readBoolean("invalidate");
 
                 if (!reader.isLastRead())
@@ -401,7 +359,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 14:
+            case 13:
                 byte plcOrd;
 
                 plcOrd = reader.readByte("plc");
@@ -413,7 +371,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 15:
+            case 14:
                 syncCommit = reader.readBoolean("syncCommit");
 
                 if (!reader.isLastRead())
@@ -421,7 +379,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 16:
+            case 15:
                 syncRollback = reader.readBoolean("syncRollback");
 
                 if (!reader.isLastRead())
@@ -429,7 +387,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 17:
+            case 16:
                 sys = reader.readBoolean("sys");
 
                 if (!reader.isLastRead())
@@ -437,7 +395,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 18:
+            case 17:
                 threadId = reader.readLong("threadId");
 
                 if (!reader.isLastRead())
@@ -445,7 +403,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage {
 
                 reader.incrementState();
 
-            case 19:
+            case 18:
                 txSize = reader.readInt("txSize");
 
                 if (!reader.isLastRead())
@@ -465,7 +423,7 @@ public class GridDistributedTxFinishRequest extends GridDistributedBaseMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 20;
+        return 19;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java
index ec02e6e..cc2783a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java
@@ -88,18 +88,6 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
     @GridDirectCollection(GridCacheVersion.class)
     private Collection<GridCacheVersion> dhtVerVals;
 
-    /** Group lock key, if any. */
-    @GridToStringInclude
-    @GridDirectTransient
-    private IgniteTxKey grpLockKey;
-
-    /** Group lock key bytes. */
-    @GridToStringExclude
-    private byte[] grpLockKeyBytes;
-
-    /** Partition lock flag. */
-    private boolean partLock;
-
     /** Expected transaction size. */
     private int txSize;
 
@@ -130,8 +118,6 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
      * @param tx Cache transaction.
      * @param reads Read entries.
      * @param writes Write entries.
-     * @param grpLockKey Group lock key.
-     * @param partLock {@code True} if preparing group-lock transaction with partition lock.
      * @param txNodes Transaction nodes mapping.
      * @param onePhaseCommit One phase commit flag.
      */
@@ -139,8 +125,6 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
         IgniteInternalTx tx,
         @Nullable Collection<IgniteTxEntry> reads,
         Collection<IgniteTxEntry> writes,
-        IgniteTxKey grpLockKey,
-        boolean partLock,
         Map<UUID, Collection<UUID>> txNodes,
         boolean onePhaseCommit
     ) {
@@ -158,8 +142,6 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
         this.reads = reads;
         this.writes = writes;
-        this.grpLockKey = grpLockKey;
-        this.partLock = partLock;
         this.txNodes = txNodes;
         this.onePhaseCommit = onePhaseCommit;
     }
@@ -272,20 +254,6 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
     }
 
     /**
-     * @return Group lock key if preparing group-lock transaction.
-     */
-    @Nullable public IgniteTxKey groupLockKey() {
-        return grpLockKey;
-    }
-
-    /**
-     * @return {@code True} if preparing group-lock transaction with partition lock.
-     */
-    public boolean partitionLock() {
-        return partLock;
-    }
-
-    /**
      * @return Expected transaction size.
      */
     public int txSize() {
@@ -310,9 +278,6 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
         if (reads != null)
             marshalTx(reads, ctx);
 
-        if (grpLockKey != null && grpLockKeyBytes == null)
-            grpLockKeyBytes = ctx.marshaller().marshal(grpLockKey);
-
         if (dhtVers != null) {
             for (IgniteTxKey key : dhtVers.keySet()) {
                 GridCacheContext cctx = ctx.cacheContext(key.cacheId());
@@ -338,9 +303,6 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
         if (reads != null)
             unmarshalTx(reads, false, ctx, ldr);
 
-        if (grpLockKeyBytes != null && grpLockKey == null)
-            grpLockKey = ctx.marshaller().unmarshal(grpLockKeyBytes, ldr);
-
         if (dhtVerKeys != null && dhtVers == null) {
             assert dhtVerVals != null;
             assert dhtVerKeys.size() == dhtVerVals.size();
@@ -397,84 +359,72 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
                 writer.incrementState();
 
             case 11:
-                if (!writer.writeByteArray("grpLockKeyBytes", grpLockKeyBytes))
-                    return false;
-
-                writer.incrementState();
-
-            case 12:
                 if (!writer.writeBoolean("invalidate", invalidate))
                     return false;
 
                 writer.incrementState();
 
-            case 13:
+            case 12:
                 if (!writer.writeByte("isolation", isolation != null ? (byte)isolation.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
 
-            case 14:
+            case 13:
                 if (!writer.writeBoolean("onePhaseCommit", onePhaseCommit))
                     return false;
 
                 writer.incrementState();
 
-            case 15:
-                if (!writer.writeBoolean("partLock", partLock))
-                    return false;
-
-                writer.incrementState();
-
-            case 16:
+            case 14:
                 if (!writer.writeByte("plc", plc != null ? (byte)plc.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
 
-            case 17:
+            case 15:
                 if (!writer.writeCollection("reads", reads, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 18:
+            case 16:
                 if (!writer.writeBoolean("sys", sys))
                     return false;
 
                 writer.incrementState();
 
-            case 19:
+            case 17:
                 if (!writer.writeLong("threadId", threadId))
                     return false;
 
                 writer.incrementState();
 
-            case 20:
+            case 18:
                 if (!writer.writeLong("timeout", timeout))
                     return false;
 
                 writer.incrementState();
 
-            case 21:
+            case 19:
                 if (!writer.writeByteArray("txNodesBytes", txNodesBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 22:
+            case 20:
                 if (!writer.writeInt("txSize", txSize))
                     return false;
 
                 writer.incrementState();
 
-            case 23:
+            case 21:
                 if (!writer.writeMessage("writeVer", writeVer))
                     return false;
 
                 writer.incrementState();
 
-            case 24:
+            case 22:
                 if (!writer.writeCollection("writes", writes, MessageCollectionItemType.MSG))
                     return false;
 
@@ -525,14 +475,6 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
                 reader.incrementState();
 
             case 11:
-                grpLockKeyBytes = reader.readByteArray("grpLockKeyBytes");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 12:
                 invalidate = reader.readBoolean("invalidate");
 
                 if (!reader.isLastRead())
@@ -540,7 +482,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 13:
+            case 12:
                 byte isolationOrd;
 
                 isolationOrd = reader.readByte("isolation");
@@ -552,7 +494,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 14:
+            case 13:
                 onePhaseCommit = reader.readBoolean("onePhaseCommit");
 
                 if (!reader.isLastRead())
@@ -560,15 +502,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 15:
-                partLock = reader.readBoolean("partLock");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 16:
+            case 14:
                 byte plcOrd;
 
                 plcOrd = reader.readByte("plc");
@@ -580,7 +514,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 17:
+            case 15:
                 reads = reader.readCollection("reads", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -588,7 +522,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 18:
+            case 16:
                 sys = reader.readBoolean("sys");
 
                 if (!reader.isLastRead())
@@ -596,7 +530,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 19:
+            case 17:
                 threadId = reader.readLong("threadId");
 
                 if (!reader.isLastRead())
@@ -604,7 +538,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 20:
+            case 18:
                 timeout = reader.readLong("timeout");
 
                 if (!reader.isLastRead())
@@ -612,7 +546,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 21:
+            case 19:
                 txNodesBytes = reader.readByteArray("txNodesBytes");
 
                 if (!reader.isLastRead())
@@ -620,7 +554,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 22:
+            case 20:
                 txSize = reader.readInt("txSize");
 
                 if (!reader.isLastRead())
@@ -628,7 +562,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 23:
+            case 21:
                 writeVer = reader.readMessage("writeVer");
 
                 if (!reader.isLastRead())
@@ -636,7 +570,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
                 reader.incrementState();
 
-            case 24:
+            case 22:
                 writes = reader.readCollection("writes", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -656,7 +590,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 25;
+        return 23;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
index 3215138..8594853 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
@@ -95,7 +95,6 @@ public class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
      * @param invalidate Invalidate flag.
      * @param timeout Timeout.
      * @param txSize Expected transaction size.
-     * @param grpLockKey Group lock key if this is a group-lock transaction.
      * @param subjId Subject ID.
      * @param taskNameHash Task name hash code.
      */
@@ -112,7 +111,6 @@ public class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
         boolean invalidate,
         long timeout,
         int txSize,
-        @Nullable IgniteTxKey grpLockKey,
         @Nullable UUID subjId,
         int taskNameHash
     ) {
@@ -128,7 +126,6 @@ public class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
             isolation,
             timeout,
             txSize,
-            grpLockKey,
             subjId,
             taskNameHash);
 
@@ -195,16 +192,6 @@ public class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
         // No-op.
     }
 
-    /**
-     * Adds group lock key to remote transaction.
-     *
-     * @param key Key.
-     */
-    public void groupLockKey(IgniteTxKey key) {
-        if (grpLockKey == null)
-            grpLockKey = key;
-    }
-
     /** {@inheritDoc} */
     @Override public GridTuple<CacheObject> peek(GridCacheContext cacheCtx,
         boolean failFast,
@@ -350,7 +337,6 @@ public class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
             entry.op(e.op());
             entry.ttl(e.ttl());
             entry.explicitVersion(e.explicitVersion());
-            entry.groupLockEntry(e.groupLockEntry());
 
             // Conflict resolution stuff.
             entry.conflictVersion(e.conflictVersion());
@@ -446,7 +432,7 @@ public class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
                         GridCacheVersion ver = txEntry.explicitVersion() != null ? txEntry.explicitVersion() : xidVer;
 
                         // If locks haven't been acquired yet, keep waiting.
-                        if (!txEntry.groupLockEntry() && !Entry.lockedBy(ver)) {
+                        if (!Entry.lockedBy(ver)) {
                             if (log.isDebugEnabled())
                                 log.debug("Transaction does not own lock for entry (will wait) [entry=" + Entry +
                                     ", tx=" + this + ']');
@@ -607,10 +593,6 @@ public class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
                                     }
                                     // No-op.
                                     else {
-                                        assert !groupLock() || txEntry.groupLockEntry() || ownsLock(txEntry.cached()):
-                                            "Transaction does not own lock for group lock entry during  commit [tx=" +
-                                                this + ", txEntry=" + txEntry + ']';
-
                                         if (conflictCtx == null || !conflictCtx.isUseOld()) {
                                             if (txEntry.ttl() != CU.TTL_NOT_CHANGED)
                                                 cached.updateTtl(null, txEntry.ttl());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
index 5b0275c..c57eded 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
@@ -833,8 +833,6 @@ public final class GridDhtLockFuture<K, V> extends GridCompoundIdentityFuture<Bo
                         cnt,
                         0,
                         inTx() ? tx.size() : cnt,
-                        inTx() ? tx.groupLockKey() : null,
-                        inTx() && tx.partitionLock(),
                         inTx() ? tx.subjectId() : null,
                         inTx() ? tx.taskNameHash() : 0,
                         read ? accessTtl : -1L,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java
index 9b69571..94ec718 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java
@@ -22,7 +22,6 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
-import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.tostring.*;
@@ -101,8 +100,6 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
      * @param dhtCnt DHT count.
      * @param nearCnt Near count.
      * @param txSize Expected transaction size.
-     * @param grpLockKey Group lock key.
-     * @param partLock {@code True} if partition lock.
      * @param subjId Subject ID.
      * @param taskNameHash Task name hash code.
      * @param accessTtl TTL for read operation.
@@ -125,8 +122,6 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
         int dhtCnt,
         int nearCnt,
         int txSize,
-        @Nullable IgniteTxKey grpLockKey,
-        boolean partLock,
         @Nullable UUID subjId,
         int taskNameHash,
         long accessTtl,
@@ -145,8 +140,6 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
             timeout,
             dhtCnt == 0 ? nearCnt : dhtCnt,
             txSize,
-            grpLockKey,
-            partLock,
             skipStore);
 
         this.topVer = topVer;
@@ -331,55 +324,55 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
         }
 
         switch (writer.state()) {
-            case 23:
+            case 21:
                 if (!writer.writeLong("accessTtl", accessTtl))
                     return false;
 
                 writer.incrementState();
 
-            case 24:
+            case 22:
                 if (!writer.writeBitSet("invalidateEntries", invalidateEntries))
                     return false;
 
                 writer.incrementState();
 
-            case 25:
+            case 23:
                 if (!writer.writeIgniteUuid("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 26:
+            case 24:
                 if (!writer.writeCollection("nearKeys", nearKeys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 27:
+            case 25:
                 if (!writer.writeByteArray("ownedBytes", ownedBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 28:
+            case 26:
                 if (!writer.writeBitSet("preloadKeys", preloadKeys))
                     return false;
 
                 writer.incrementState();
 
-            case 29:
+            case 27:
                 if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
-            case 30:
+            case 28:
                 if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
-            case 31:
+            case 29:
                 if (!writer.writeMessage("topVer", topVer))
                     return false;
 
@@ -401,7 +394,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
             return false;
 
         switch (reader.state()) {
-            case 23:
+            case 21:
                 accessTtl = reader.readLong("accessTtl");
 
                 if (!reader.isLastRead())
@@ -409,7 +402,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 24:
+            case 22:
                 invalidateEntries = reader.readBitSet("invalidateEntries");
 
                 if (!reader.isLastRead())
@@ -417,7 +410,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 25:
+            case 23:
                 miniId = reader.readIgniteUuid("miniId");
 
                 if (!reader.isLastRead())
@@ -425,7 +418,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 26:
+            case 24:
                 nearKeys = reader.readCollection("nearKeys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -433,7 +426,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 27:
+            case 25:
                 ownedBytes = reader.readByteArray("ownedBytes");
 
                 if (!reader.isLastRead())
@@ -441,7 +434,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 28:
+            case 26:
                 preloadKeys = reader.readBitSet("preloadKeys");
 
                 if (!reader.isLastRead())
@@ -449,7 +442,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 29:
+            case 27:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -457,7 +450,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 30:
+            case 28:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -465,7 +458,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 31:
+            case 29:
                 topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
@@ -485,7 +478,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 32;
+        return 30;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
index 068e8b2..26eef50 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
@@ -203,7 +203,6 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
                                     req.isInvalidate(),
                                     req.timeout(),
                                     req.txSize(),
-                                    req.groupLockKey(),
                                     req.subjectId(),
                                     req.taskNameHash());
 
@@ -222,9 +221,6 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
                                 null,
                                 req.accessTtl(),
                                 req.skipStore());
-
-                            if (req.groupLock())
-                                tx.groupLockKey(txKey);
                         }
 
                         entry = entryExx(key, req.topologyVersion());
@@ -810,8 +806,6 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
                                     req.isInvalidate(),
                                     false,
                                     req.txSize(),
-                                    req.groupLockKey(),
-                                    req.partitionLock(),
                                     null,
                                     req.subjectId(),
                                     req.taskNameHash());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
index 7c35fc5..7fd79e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
@@ -309,7 +309,6 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
                 tx.rolledbackVersions(),
                 tx.pendingVersions(),
                 tx.size(),
-                tx.groupLockKey(),
                 tx.subjectId(),
                 tx.taskNameHash());
 
@@ -387,7 +386,6 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
                 tx.rolledbackVersions(),
                 tx.pendingVersions(),
                 tx.size(),
-                tx.groupLockKey(),
                 tx.subjectId(),
                 tx.taskNameHash());
 
@@ -439,7 +437,6 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
                     tx.rolledbackVersions(),
                     tx.pendingVersions(),
                     tx.size(),
-                    tx.groupLockKey(),
                     tx.subjectId(),
                     tx.taskNameHash());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java
index d20a7c3..7b077c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java
@@ -21,7 +21,6 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
-import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -97,7 +96,6 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
      * @param rolledbackVers Rolled back versions.
      * @param pendingVers Pending versions.
      * @param txSize Expected transaction size.
-     * @param grpLockKey Group lock key.
      * @param subjId Subject ID.
      * @param taskNameHash Task name hash.
      */
@@ -122,12 +120,11 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
         Collection<GridCacheVersion> rolledbackVers,
         Collection<GridCacheVersion> pendingVers,
         int txSize,
-        @Nullable IgniteTxKey grpLockKey,
         @Nullable UUID subjId,
         int taskNameHash
     ) {
         super(xidVer, futId, commitVer, threadId, commit, invalidate, sys, plc, syncCommit, syncRollback, baseVer,
-            committedVers, rolledbackVers, txSize, grpLockKey);
+            committedVers, rolledbackVers, txSize);
 
         assert miniId != null;
         assert nearNodeId != null;
@@ -241,55 +238,55 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
         }
 
         switch (writer.state()) {
-            case 20:
+            case 19:
                 if (!writer.writeByte("isolation", isolation != null ? (byte)isolation.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
 
-            case 21:
+            case 20:
                 if (!writer.writeIgniteUuid("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 22:
+            case 21:
                 if (!writer.writeUuid("nearNodeId", nearNodeId))
                     return false;
 
                 writer.incrementState();
 
-            case 23:
+            case 22:
                 if (!writer.writeCollection("pendingVers", pendingVers, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 24:
+            case 23:
                 if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
-            case 25:
+            case 24:
                 if (!writer.writeBoolean("sysInvalidate", sysInvalidate))
                     return false;
 
                 writer.incrementState();
 
-            case 26:
+            case 25:
                 if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
-            case 27:
+            case 26:
                 if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
-            case 28:
+            case 27:
                 if (!writer.writeMessage("writeVer", writeVer))
                     return false;
 
@@ -311,7 +308,7 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
             return false;
 
         switch (reader.state()) {
-            case 20:
+            case 19:
                 byte isolationOrd;
 
                 isolationOrd = reader.readByte("isolation");
@@ -323,7 +320,7 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
 
                 reader.incrementState();
 
-            case 21:
+            case 20:
                 miniId = reader.readIgniteUuid("miniId");
 
                 if (!reader.isLastRead())
@@ -331,7 +328,7 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
 
                 reader.incrementState();
 
-            case 22:
+            case 21:
                 nearNodeId = reader.readUuid("nearNodeId");
 
                 if (!reader.isLastRead())
@@ -339,7 +336,7 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
 
                 reader.incrementState();
 
-            case 23:
+            case 22:
                 pendingVers = reader.readCollection("pendingVers", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -347,7 +344,7 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
 
                 reader.incrementState();
 
-            case 24:
+            case 23:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -355,7 +352,7 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
 
                 reader.incrementState();
 
-            case 25:
+            case 24:
                 sysInvalidate = reader.readBoolean("sysInvalidate");
 
                 if (!reader.isLastRead())
@@ -363,7 +360,7 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
 
                 reader.incrementState();
 
-            case 26:
+            case 25:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -371,7 +368,7 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
 
                 reader.incrementState();
 
-            case 27:
+            case 26:
                 topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
@@ -379,7 +376,7 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
 
                 reader.incrementState();
 
-            case 28:
+            case 27:
                 writeVer = reader.readMessage("writeVer");
 
                 if (!reader.isLastRead())
@@ -399,6 +396,6 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 29;
+        return 28;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
index 614f520..841cac8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
@@ -93,8 +93,6 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
      * @param timeout Timeout.
      * @param storeEnabled Store enabled flag.
      * @param txSize Expected transaction size.
-     * @param grpLockKey Group lock key if this is a group-lock transaction.
-     * @param partLock {@code True} if this is a group-lock transaction and whole partition should be locked.
      * @param txNodes Transaction nodes mapping.
      */
     public GridDhtTxLocal(
@@ -115,8 +113,6 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
         boolean invalidate,
         boolean storeEnabled,
         int txSize,
-        @Nullable IgniteTxKey grpLockKey,
-        boolean partLock,
         Map<UUID, Collection<UUID>> txNodes,
         UUID subjId,
         int taskNameHash
@@ -135,8 +131,6 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
             invalidate,
             storeEnabled,
             txSize,
-            grpLockKey,
-            partLock,
             subjId,
             taskNameHash);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
index 444085f..54b59b8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
@@ -94,8 +94,6 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
      * @param isolation Isolation.
      * @param timeout Timeout.
      * @param txSize Expected transaction size.
-     * @param grpLockKey Group lock key if this is a group-lock transaction.
-     * @param partLock If this is a group-lock transaction and the whole partition should be locked.
      */
     protected GridDhtTxLocalAdapter(
         GridCacheSharedContext cctx,
@@ -111,13 +109,11 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
         boolean invalidate,
         boolean storeEnabled,
         int txSize,
-        @Nullable IgniteTxKey grpLockKey,
-        boolean partLock,
         @Nullable UUID subjId,
         int taskNameHash
     ) {
         super(cctx, xidVer, implicit, implicitSingle, sys, plc, concurrency, isolation, timeout, invalidate,
-            storeEnabled, txSize, grpLockKey, partLock, subjId, taskNameHash);
+            storeEnabled, txSize, subjId, taskNameHash);
 
         assert cctx != null;
 
@@ -733,68 +729,6 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override protected void addGroupTxMapping(Collection<IgniteTxKey> keys) {
-        assert groupLock();
-
-        for (GridDistributedTxMapping mapping : dhtMap.values())
-            mapping.entries(Collections.unmodifiableCollection(txMap.values()), true);
-
-        // Here we know that affinity key for all given keys is our group lock key.
-        // Just add entries to dht mapping.
-        // Add near readers. If near cache is disabled on all nodes, do nothing.
-        Collection<UUID> backupIds = dhtMap.keySet();
-
-        Map<ClusterNode, List<GridDhtCacheEntry>> locNearMap = null;
-
-        for (IgniteTxKey key : keys) {
-            IgniteTxEntry txEntry = entry(key);
-
-            if (!txEntry.groupLockEntry() || txEntry.context().isNear())
-                continue;
-
-            assert txEntry.cached() instanceof GridDhtCacheEntry : "Invalid entry type: " + txEntry.cached();
-
-            while (true) {
-                try {
-                    GridDhtCacheEntry entry = (GridDhtCacheEntry)txEntry.cached();
-
-                    Collection<UUID> readers = entry.readers();
-
-                    if (!F.isEmpty(readers)) {
-                        Collection<ClusterNode> nearNodes = cctx.discovery().nodes(readers, F0.notEqualTo(nearNodeId()),
-                            F.notIn(backupIds));
-
-                        if (log.isDebugEnabled())
-                            log.debug("Mapping entry to near nodes [nodes=" + U.nodeIds(nearNodes) + ", entry=" +
-                                entry + ']');
-
-                        for (ClusterNode n : nearNodes) {
-                            if (locNearMap == null)
-                                locNearMap = new HashMap<>();
-
-                            List<GridDhtCacheEntry> entries = locNearMap.get(n);
-
-                            if (entries == null)
-                                locNearMap.put(n, entries = new LinkedList<>());
-
-                            entries.add(entry);
-                        }
-                    }
-
-                    break;
-                }
-                catch (GridCacheEntryRemovedException ignored) {
-                    // Retry.
-                    txEntry.cached(txEntry.context().dht().entryExx(key.key(), topologyVersion()));
-                }
-            }
-        }
-
-        if (locNearMap != null)
-            addNearNodeEntryMapping(locNearMap);
-    }
-
-    /** {@inheritDoc} */
     @SuppressWarnings({"CatchGenericClass", "ThrowableInstanceNeverThrown"})
     @Override public boolean finish(boolean commit) throws IgniteCheckedException {
         if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/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 0e64726..3056ae5 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
@@ -414,8 +414,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
         if (log.isDebugEnabled())
             log.debug("Marking all local candidates as ready: " + this);
 
-        Iterable<IgniteTxEntry> checkEntries = tx.groupLock() ?
-            Collections.singletonList(tx.groupLockEntry()) : writes;
+        Iterable<IgniteTxEntry> checkEntries = writes;
 
         for (IgniteTxEntry txEntry : checkEntries) {
             GridCacheContext cacheCtx = txEntry.context();
@@ -431,10 +430,8 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                 txEntry.cached(entry);
             }
 
-            if (tx.optimistic() && txEntry.explicitVersion() == null) {
-                if (!tx.groupLock() || tx.groupLockKey().equals(entry.txKey()))
-                    lockKeys.add(txEntry.txKey());
-            }
+            if (tx.optimistic() && txEntry.explicitVersion() == null)
+                lockKeys.add(txEntry.txKey());
 
             while (true) {
                 try {
@@ -803,8 +800,6 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                         tx,
                         dhtWrites,
                         nearWrites,
-                        tx.groupLockKey(),
-                        tx.partitionLock(),
                         txNodes,
                         tx.nearXidVersion(),
                         true,
@@ -823,9 +818,6 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                             if (entry.explicitVersion() == null) {
                                 GridCacheMvccCandidate added = cached.candidate(version());
 
-                                assert added != null || entry.groupLockEntry() :
-                                    "Null candidate for non-group-lock entry " +
-                                        "[added=" + added + ", entry=" + entry + ']';
                                 assert added == null || added.dhtLocal() :
                                     "Got non-dht-local candidate for prepare future " +
                                         "[added=" + added + ", entry=" + entry + ']';
@@ -906,8 +898,6 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                             tx,
                             null,
                             nearMapping.writes(),
-                            tx.groupLockKey(),
-                            tx.partitionLock(),
                             tx.transactionNodes(),
                             tx.nearXidVersion(),
                             true,
@@ -920,8 +910,6 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                                 if (entry.explicitVersion() == null) {
                                     GridCacheMvccCandidate added = entry.cached().candidate(version());
 
-                                    assert added != null || entry.groupLockEntry() : "Null candidate for non-group-lock entry " +
-                                        "[added=" + added + ", entry=" + entry + ']';
                                     assert added == null || added.dhtLocal() : "Got non-dht-local candidate for prepare future" +
                                         "[added=" + added + ", entry=" + entry + ']';
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
index c033273..73f86fd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
@@ -104,8 +104,6 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
      * @param tx Transaction.
      * @param dhtWrites DHT writes.
      * @param nearWrites Near writes.
-     * @param grpLockKey Group lock key if preparing group-lock transaction.
-     * @param partLock {@code True} if group-lock transaction locks partition.
      * @param txNodes Transaction nodes mapping.
      * @param nearXidVer Near transaction ID.
      * @param last {@code True} if this is last prepare request for node.
@@ -118,15 +116,13 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
         GridDhtTxLocalAdapter tx,
         Collection<IgniteTxEntry> dhtWrites,
         Collection<IgniteTxEntry> nearWrites,
-        IgniteTxKey grpLockKey,
-        boolean partLock,
         Map<UUID, Collection<UUID>> txNodes,
         GridCacheVersion nearXidVer,
         boolean last,
         boolean onePhaseCommit,
         UUID subjId,
         int taskNameHash) {
-        super(tx, null, dhtWrites, grpLockKey, partLock, txNodes, onePhaseCommit);
+        super(tx, null, dhtWrites, txNodes, onePhaseCommit);
 
         assert futId != null;
         assert miniId != null;
@@ -337,79 +333,79 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
         }
 
         switch (writer.state()) {
-            case 25:
+            case 23:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 26:
+            case 24:
                 if (!writer.writeBitSet("invalidateNearEntries", invalidateNearEntries))
                     return false;
 
                 writer.incrementState();
 
-            case 27:
+            case 25:
                 if (!writer.writeBoolean("last", last))
                     return false;
 
                 writer.incrementState();
 
-            case 28:
+            case 26:
                 if (!writer.writeIgniteUuid("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 29:
+            case 27:
                 if (!writer.writeUuid("nearNodeId", nearNodeId))
                     return false;
 
                 writer.incrementState();
 
-            case 30:
+            case 28:
                 if (!writer.writeCollection("nearWrites", nearWrites, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 31:
+            case 29:
                 if (!writer.writeMessage("nearXidVer", nearXidVer))
                     return false;
 
                 writer.incrementState();
 
-            case 32:
+            case 30:
                 if (!writer.writeCollection("ownedKeys", ownedKeys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 33:
+            case 31:
                 if (!writer.writeCollection("ownedVals", ownedVals, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 34:
+            case 32:
                 if (!writer.writeBitSet("preloadKeys", preloadKeys))
                     return false;
 
                 writer.incrementState();
 
-            case 35:
+            case 33:
                 if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
-            case 36:
+            case 34:
                 if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
-            case 37:
+            case 35:
                 if (!writer.writeMessage("topVer", topVer))
                     return false;
 
@@ -431,7 +427,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
             return false;
 
         switch (reader.state()) {
-            case 25:
+            case 23:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -439,7 +435,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 26:
+            case 24:
                 invalidateNearEntries = reader.readBitSet("invalidateNearEntries");
 
                 if (!reader.isLastRead())
@@ -447,7 +443,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 27:
+            case 25:
                 last = reader.readBoolean("last");
 
                 if (!reader.isLastRead())
@@ -455,7 +451,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 28:
+            case 26:
                 miniId = reader.readIgniteUuid("miniId");
 
                 if (!reader.isLastRead())
@@ -463,7 +459,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 29:
+            case 27:
                 nearNodeId = reader.readUuid("nearNodeId");
 
                 if (!reader.isLastRead())
@@ -471,7 +467,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 30:
+            case 28:
                 nearWrites = reader.readCollection("nearWrites", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -479,7 +475,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 31:
+            case 29:
                 nearXidVer = reader.readMessage("nearXidVer");
 
                 if (!reader.isLastRead())
@@ -487,7 +483,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 32:
+            case 30:
                 ownedKeys = reader.readCollection("ownedKeys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -495,7 +491,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 33:
+            case 31:
                 ownedVals = reader.readCollection("ownedVals", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -503,7 +499,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 34:
+            case 32:
                 preloadKeys = reader.readBitSet("preloadKeys");
 
                 if (!reader.isLastRead())
@@ -511,7 +507,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 35:
+            case 33:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -519,7 +515,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 36:
+            case 34:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -527,7 +523,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 37:
+            case 35:
                 topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
@@ -547,6 +543,6 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 38;
+        return 36;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java
index 30464a5..0a69910 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java
@@ -77,7 +77,6 @@ public class GridDhtTxRemote extends GridDistributedTxRemoteAdapter {
      * @param timeout Timeout.
      * @param ctx Cache context.
      * @param txSize Expected transaction size.
-     * @param grpLockKey Group lock key if this is a group-lock transaction.
      * @param nearXidVer Near transaction ID.
      * @param txNodes Transaction nodes mapping.
      */
@@ -97,14 +96,13 @@ public class GridDhtTxRemote extends GridDistributedTxRemoteAdapter {
         boolean invalidate,
         long timeout,
         int txSize,
-        @Nullable IgniteTxKey grpLockKey,
         GridCacheVersion nearXidVer,
         Map<UUID, Collection<UUID>> txNodes,
         @Nullable UUID subjId,
         int taskNameHash
     ) {
         super(ctx, nodeId, rmtThreadId, xidVer, commitVer, sys, plc, concurrency, isolation, invalidate, timeout,
-            txSize, grpLockKey, subjId, taskNameHash);
+            txSize, subjId, taskNameHash);
 
         assert nearNodeId != null;
         assert rmtFutId != null;
@@ -139,7 +137,6 @@ public class GridDhtTxRemote extends GridDistributedTxRemoteAdapter {
      * @param timeout Timeout.
      * @param ctx Cache context.
      * @param txSize Expected transaction size.
-     * @param grpLockKey Group lock key if transaction is group-lock.
      */
     public GridDhtTxRemote(
         GridCacheSharedContext ctx,
@@ -158,12 +155,11 @@ public class GridDhtTxRemote extends GridDistributedTxRemoteAdapter {
         boolean invalidate,
         long timeout,
         int txSize,
-        @Nullable IgniteTxKey grpLockKey,
         @Nullable UUID subjId,
         int taskNameHash
     ) {
         super(ctx, nodeId, rmtThreadId, xidVer, commitVer, sys, plc, concurrency, isolation, invalidate, timeout,
-            txSize, grpLockKey, subjId, taskNameHash);
+            txSize, subjId, taskNameHash);
 
         assert nearNodeId != null;
         assert rmtFutId != null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
index e905cd5..e6a4eaf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
@@ -754,8 +754,6 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
                                         mappedKeys.size(),
                                         inTx() ? tx.size() : mappedKeys.size(),
                                         inTx() && tx.syncCommit(),
-                                        inTx() ? tx.groupLockKey() : null,
-                                        inTx() && tx.partitionLock(),
                                         inTx() ? tx.subjectId() : null,
                                         inTx() ? tx.taskNameHash() : 0,
                                         read ? accessTtl : -1L,
@@ -1090,10 +1088,6 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
             // If primary node left the grid before lock acquisition, fail the whole future.
             throw newTopologyException(null, primary.id());
 
-        if (inTx() && tx.groupLock() && !primary.isLocal())
-            throw new IgniteCheckedException("Failed to start group lock transaction (local node is not primary for " +
-                " key) [key=" + key + ", primaryNodeId=" + primary.id() + ']');
-
         if (mapping == null || !primary.id().equals(mapping.node().id()))
             mapping = new GridNearLockMapping(primary, key);
         else

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
index 25bd76b..0ffb4e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
@@ -890,8 +890,6 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
                                             mappedKeys.size(),
                                             inTx() ? tx.size() : mappedKeys.size(),
                                             inTx() && tx.syncCommit(),
-                                            inTx() ? tx.groupLockKey() : null,
-                                            inTx() && tx.partitionLock(),
                                             inTx() ? tx.subjectId() : null,
                                             inTx() ? tx.taskNameHash() : 0,
                                             read ? accessTtl : -1L,
@@ -1188,10 +1186,6 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
             // If primary node left the grid before lock acquisition, fail the whole future.
             throw newTopologyException(null, primary.id());
 
-        if (inTx() && tx.groupLock() && !primary.isLocal())
-            throw new IgniteCheckedException("Failed to start group lock transaction (local node is not primary for " +
-                " key) [key=" + key + ", primaryNodeId=" + primary.id() + ']');
-
         if (mapping == null || !primary.id().equals(mapping.node().id()))
             mapping = new GridNearLockMapping(primary, key);
         else