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

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

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/GridNearLockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java
index 1ba4bfe..e71dd65 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java
@@ -21,7 +21,6 @@ import org.apache.ignite.*;
 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.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -105,8 +104,6 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
      * @param keyCnt Number of keys.
      * @param txSize Expected transaction size.
      * @param syncCommit Synchronous commit flag.
-     * @param grpLockKey Group lock key if this is a group-lock transaction.
-     * @param partLock If partition is locked.
      * @param subjId Subject ID.
      * @param taskNameHash Task name hash code.
      * @param accessTtl TTL for read operation.
@@ -130,8 +127,6 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
         int keyCnt,
         int txSize,
         boolean syncCommit,
-        @Nullable IgniteTxKey grpLockKey,
-        boolean partLock,
         @Nullable UUID subjId,
         int taskNameHash,
         long accessTtl,
@@ -151,8 +146,6 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
             timeout,
             keyCnt,
             txSize,
-            grpLockKey,
-            partLock,
             skipStore);
 
         assert topVer.compareTo(AffinityTopologyVersion.ZERO) > 0;
@@ -356,79 +349,79 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
         }
 
         switch (writer.state()) {
-            case 23:
+            case 21:
                 if (!writer.writeLong("accessTtl", accessTtl))
                     return false;
 
                 writer.incrementState();
 
-            case 24:
+            case 22:
                 if (!writer.writeObjectArray("dhtVers", dhtVers, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 25:
+            case 23:
                 if (!writer.writeObjectArray("filter", filter, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
-            case 26:
+            case 24:
                 if (!writer.writeBoolean("hasTransforms", hasTransforms))
                     return false;
 
                 writer.incrementState();
 
-            case 27:
+            case 25:
                 if (!writer.writeBoolean("implicitSingleTx", implicitSingleTx))
                     return false;
 
                 writer.incrementState();
 
-            case 28:
+            case 26:
                 if (!writer.writeBoolean("implicitTx", implicitTx))
                     return false;
 
                 writer.incrementState();
 
-            case 29:
+            case 27:
                 if (!writer.writeIgniteUuid("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 30:
+            case 28:
                 if (!writer.writeBoolean("onePhaseCommit", onePhaseCommit))
                     return false;
 
                 writer.incrementState();
 
-            case 31:
+            case 29:
                 if (!writer.writeBoolean("retVal", retVal))
                     return false;
 
                 writer.incrementState();
 
-            case 32:
+            case 30:
                 if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
-            case 33:
+            case 31:
                 if (!writer.writeBoolean("syncCommit", syncCommit))
                     return false;
 
                 writer.incrementState();
 
-            case 34:
+            case 32:
                 if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
-            case 35:
+            case 33:
                 if (!writer.writeMessage("topVer", topVer))
                     return false;
 
@@ -450,7 +443,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
             return false;
 
         switch (reader.state()) {
-            case 23:
+            case 21:
                 accessTtl = reader.readLong("accessTtl");
 
                 if (!reader.isLastRead())
@@ -458,7 +451,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 24:
+            case 22:
                 dhtVers = reader.readObjectArray("dhtVers", MessageCollectionItemType.MSG, GridCacheVersion.class);
 
                 if (!reader.isLastRead())
@@ -466,7 +459,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 25:
+            case 23:
                 filter = reader.readObjectArray("filter", MessageCollectionItemType.MSG, CacheEntryPredicate.class);
 
                 if (!reader.isLastRead())
@@ -474,7 +467,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 26:
+            case 24:
                 hasTransforms = reader.readBoolean("hasTransforms");
 
                 if (!reader.isLastRead())
@@ -482,7 +475,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 27:
+            case 25:
                 implicitSingleTx = reader.readBoolean("implicitSingleTx");
 
                 if (!reader.isLastRead())
@@ -490,7 +483,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 28:
+            case 26:
                 implicitTx = reader.readBoolean("implicitTx");
 
                 if (!reader.isLastRead())
@@ -498,7 +491,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 29:
+            case 27:
                 miniId = reader.readIgniteUuid("miniId");
 
                 if (!reader.isLastRead())
@@ -506,7 +499,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 30:
+            case 28:
                 onePhaseCommit = reader.readBoolean("onePhaseCommit");
 
                 if (!reader.isLastRead())
@@ -514,7 +507,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 31:
+            case 29:
                 retVal = reader.readBoolean("retVal");
 
                 if (!reader.isLastRead())
@@ -522,7 +515,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 32:
+            case 30:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -530,7 +523,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 33:
+            case 31:
                 syncCommit = reader.readBoolean("syncCommit");
 
                 if (!reader.isLastRead())
@@ -538,7 +531,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 34:
+            case 32:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -546,7 +539,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 35:
+            case 33:
                 topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
@@ -566,7 +559,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 36;
+        return 34;
     }
 
     /** {@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/near/GridNearOptimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
index 51c7ccd..4f74303 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
@@ -493,8 +493,6 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
             tx,
             tx.optimistic() && tx.serializable() ? m.reads() : null,
             m.writes(),
-            tx.groupLockKey(),
-            tx.partitionLock(),
             m.near(),
             txMapping.transactionNodes(),
             m.last(),
@@ -548,9 +546,6 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
             });
         }
         else {
-            assert !tx.groupLock() : "Got group lock transaction that is mapped on remote node [tx=" + tx +
-                ", nodeId=" + n.id() + ']';
-
             try {
                 cctx.io().send(n, req, tx.ioPolicy());
             }
@@ -590,10 +585,6 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
                 ", primary=" + U.toShortString(primary) + ", topVer=" + topVer + ']');
         }
 
-        if (tx.groupLock() && !primary.isLocal())
-            throw new IgniteCheckedException("Failed to prepare group lock transaction (local node is not primary for " +
-                " key)[key=" + entry.key() + ", primaryNodeId=" + primary.id() + ']');
-
         // Must re-initialize cached entry while holding topology lock.
         if (cacheCtx.isNear())
             entry.cached(cacheCtx.nearTx().entryExx(entry.key(), topVer));
@@ -603,10 +594,8 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
             entry.cached(cacheCtx.local().entryEx(entry.key(), topVer));
 
         if (cacheCtx.isNear() || cacheCtx.isLocal()) {
-            if (waitLock && entry.explicitVersion() == null) {
-                if (!tx.groupLock() || tx.groupLockKey().equals(entry.txKey()))
-                    lockKeys.add(entry.txKey());
-            }
+            if (waitLock && entry.explicitVersion() == null)
+                lockKeys.add(entry.txKey());
         }
 
         if (cur == null || !cur.node().id().equals(primary.id()) || cur.near() != cacheCtx.isNear()) {

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/GridNearPessimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
index 998df9e..bce62c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
@@ -178,8 +178,6 @@ public class GridNearPessimisticTxPrepareFuture extends GridNearTxPrepareFutureA
                 tx,
                 m.reads(),
                 m.writes(),
-                /*grp lock key*/null,
-                /*part lock*/false,
                 m.near(),
                 txMapping.transactionNodes(),
                 true,

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/GridNearTransactionalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
index 581c7e0..df7a65f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
@@ -301,14 +301,10 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
                                         req.isInvalidate(),
                                         req.timeout(),
                                         req.txSize(),
-                                        req.groupLockKey(),
                                         req.subjectId(),
                                         req.taskNameHash()
                                     );
 
-                                    if (req.groupLock())
-                                        tx.groupLockKey(txKey);
-
                                     tx = ctx.tm().onCreated(null, tx);
 
                                     if (tx == null || !ctx.tm().onStarted(tx))

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/GridNearTxFinishRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java
index 7b0b811..b44f821 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java
@@ -97,7 +97,7 @@ public class GridNearTxFinishRequest extends GridDistributedTxFinishRequest {
         @Nullable UUID subjId,
         int taskNameHash) {
         super(xidVer, futId, null, threadId, commit, invalidate, sys, plc, syncCommit, syncRollback, baseVer,
-            committedVers, rolledbackVers, txSize, null);
+            committedVers, rolledbackVers, txSize);
 
         this.explicitLock = explicitLock;
         this.storeEnabled = storeEnabled;
@@ -170,37 +170,37 @@ public class GridNearTxFinishRequest extends GridDistributedTxFinishRequest {
         }
 
         switch (writer.state()) {
-            case 20:
+            case 19:
                 if (!writer.writeBoolean("explicitLock", explicitLock))
                     return false;
 
                 writer.incrementState();
 
-            case 21:
+            case 20:
                 if (!writer.writeIgniteUuid("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 22:
+            case 21:
                 if (!writer.writeBoolean("storeEnabled", storeEnabled))
                     return false;
 
                 writer.incrementState();
 
-            case 23:
+            case 22:
                 if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
-            case 24:
+            case 23:
                 if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
-            case 25:
+            case 24:
                 if (!writer.writeMessage("topVer", topVer))
                     return false;
 
@@ -222,7 +222,7 @@ public class GridNearTxFinishRequest extends GridDistributedTxFinishRequest {
             return false;
 
         switch (reader.state()) {
-            case 20:
+            case 19:
                 explicitLock = reader.readBoolean("explicitLock");
 
                 if (!reader.isLastRead())
@@ -230,7 +230,7 @@ public class GridNearTxFinishRequest extends GridDistributedTxFinishRequest {
 
                 reader.incrementState();
 
-            case 21:
+            case 20:
                 miniId = reader.readIgniteUuid("miniId");
 
                 if (!reader.isLastRead())
@@ -238,7 +238,7 @@ public class GridNearTxFinishRequest extends GridDistributedTxFinishRequest {
 
                 reader.incrementState();
 
-            case 22:
+            case 21:
                 storeEnabled = reader.readBoolean("storeEnabled");
 
                 if (!reader.isLastRead())
@@ -246,7 +246,7 @@ public class GridNearTxFinishRequest extends GridDistributedTxFinishRequest {
 
                 reader.incrementState();
 
-            case 23:
+            case 22:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -254,7 +254,7 @@ public class GridNearTxFinishRequest extends GridDistributedTxFinishRequest {
 
                 reader.incrementState();
 
-            case 24:
+            case 23:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -262,7 +262,7 @@ public class GridNearTxFinishRequest extends GridDistributedTxFinishRequest {
 
                 reader.incrementState();
 
-            case 25:
+            case 24:
                 topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
@@ -282,7 +282,7 @@ public class GridNearTxFinishRequest extends GridDistributedTxFinishRequest {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 26;
+        return 25;
     }
 
     /** {@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/near/GridNearTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index 50d3f3e..5c426ed 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -102,8 +102,6 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
      * @param timeout Timeout.
      * @param storeEnabled Store enabled flag.
      * @param txSize 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 the whole partition should be locked.
      * @param subjId Subject ID.
      * @param taskNameHash Task name hash code.
      */
@@ -118,8 +116,6 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
         long timeout,
         boolean storeEnabled,
         int txSize,
-        @Nullable IgniteTxKey grpLockKey,
-        boolean partLock,
         @Nullable UUID subjId,
         int taskNameHash
     ) {
@@ -137,8 +133,6 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
             false,
             storeEnabled,
             txSize,
-            grpLockKey,
-            partLock,
             subjId,
             taskNameHash);
 
@@ -272,9 +266,6 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
 
     /** {@inheritDoc} */
     @Override public Collection<IgniteTxEntry> optimisticLockEntries() {
-        if (groupLock())
-            return super.optimisticLockEntries();
-
         return optimisticLockEntries;
     }
 
@@ -416,13 +407,6 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
         }
     }
 
-    /** {@inheritDoc} */
-    @Override protected void addGroupTxMapping(Collection<IgniteTxKey> keys) {
-        super.addGroupTxMapping(keys);
-
-        addKeyMapping(cctx.localNode(), keys);
-    }
-
     /**
      * Adds key mapping to dht mapping.
      *
@@ -562,9 +546,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
         Collection<GridCacheVersion> committedVers,
         Collection<GridCacheVersion> rolledbackVers)
     {
-        Collection<IgniteTxEntry> entries = groupLock() ?
-            Collections.singletonList(groupLockEntry()) :
-            F.concat(false, mapping.reads(), mapping.writes());
+        Collection<IgniteTxEntry> entries = F.concat(false, mapping.reads(), mapping.writes());
 
         for (IgniteTxEntry txEntry : entries) {
             while (true) {

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/GridNearTxPrepareRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
index f0587ac..a08637d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
@@ -88,8 +88,6 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
      * @param tx Transaction.
      * @param reads Read entries.
      * @param writes Write entries.
-     * @param grpLockKey Group lock key if preparing group-lock transaction.
-     * @param partLock {@code True} if preparing group-lock transaction with partition lock.
      * @param near {@code True} if mapping is for near caches.
      * @param txNodes Transaction nodes mapping.
      * @param last {@code True} if this last prepare request for node.
@@ -103,8 +101,6 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
         IgniteInternalTx tx,
         Collection<IgniteTxEntry> reads,
         Collection<IgniteTxEntry> writes,
-        IgniteTxKey grpLockKey,
-        boolean partLock,
         boolean near,
         Map<UUID, Collection<UUID>> txNodes,
         boolean last,
@@ -116,7 +112,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
         @Nullable UUID subjId,
         int taskNameHash
     ) {
-        super(tx, reads, writes, grpLockKey, partLock, txNodes, onePhaseCommit);
+        super(tx, reads, writes, txNodes, onePhaseCommit);
 
         assert futId != null;
 
@@ -270,67 +266,67 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
         }
 
         switch (writer.state()) {
-            case 25:
+            case 23:
                 if (!writer.writeBoolean("explicitLock", explicitLock))
                     return false;
 
                 writer.incrementState();
 
-            case 26:
+            case 24:
                 if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
-            case 27:
+            case 25:
                 if (!writer.writeBoolean("implicitSingle", implicitSingle))
                     return false;
 
                 writer.incrementState();
 
-            case 28:
+            case 26:
                 if (!writer.writeBoolean("last", last))
                     return false;
 
                 writer.incrementState();
 
-            case 29:
+            case 27:
                 if (!writer.writeCollection("lastBackups", lastBackups, MessageCollectionItemType.UUID))
                     return false;
 
                 writer.incrementState();
 
-            case 30:
+            case 28:
                 if (!writer.writeIgniteUuid("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 31:
+            case 29:
                 if (!writer.writeBoolean("near", near))
                     return false;
 
                 writer.incrementState();
 
-            case 32:
+            case 30:
                 if (!writer.writeBoolean("retVal", retVal))
                     return false;
 
                 writer.incrementState();
 
-            case 33:
+            case 31:
                 if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
-            case 34:
+            case 32:
                 if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
-            case 35:
+            case 33:
                 if (!writer.writeMessage("topVer", topVer))
                     return false;
 
@@ -352,7 +348,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
             return false;
 
         switch (reader.state()) {
-            case 25:
+            case 23:
                 explicitLock = reader.readBoolean("explicitLock");
 
                 if (!reader.isLastRead())
@@ -360,7 +356,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 26:
+            case 24:
                 futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
@@ -368,7 +364,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 27:
+            case 25:
                 implicitSingle = reader.readBoolean("implicitSingle");
 
                 if (!reader.isLastRead())
@@ -376,7 +372,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 28:
+            case 26:
                 last = reader.readBoolean("last");
 
                 if (!reader.isLastRead())
@@ -384,7 +380,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 29:
+            case 27:
                 lastBackups = reader.readCollection("lastBackups", MessageCollectionItemType.UUID);
 
                 if (!reader.isLastRead())
@@ -392,7 +388,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 30:
+            case 28:
                 miniId = reader.readIgniteUuid("miniId");
 
                 if (!reader.isLastRead())
@@ -400,7 +396,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 31:
+            case 29:
                 near = reader.readBoolean("near");
 
                 if (!reader.isLastRead())
@@ -408,7 +404,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 32:
+            case 30:
                 retVal = reader.readBoolean("retVal");
 
                 if (!reader.isLastRead())
@@ -416,7 +412,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 33:
+            case 31:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -424,7 +420,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 34:
+            case 32:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -432,7 +428,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
                 reader.incrementState();
 
-            case 35:
+            case 33:
                 topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
@@ -452,7 +448,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 36;
+        return 34;
     }
 
     /** {@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/near/GridNearTxRemote.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxRemote.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxRemote.java
index b6b6017..49283cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxRemote.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxRemote.java
@@ -51,9 +51,6 @@ public class GridNearTxRemote extends GridDistributedTxRemoteAdapter {
     /** Owned versions. */
     private Map<IgniteTxKey, GridCacheVersion> owned;
 
-    /** Group lock flag. */
-    private boolean grpLock;
-
     /**
      * Empty constructor required for {@link Externalizable}.
      */
@@ -78,7 +75,6 @@ public class GridNearTxRemote extends GridDistributedTxRemoteAdapter {
      * @param writeEntries Write entries.
      * @param ctx Cache registry.
      * @param txSize Expected transaction size.
-     * @param grpLockKey Group lock key if this is a group-lock transaction.
      * @throws IgniteCheckedException If unmarshalling failed.
      */
     public GridNearTxRemote(
@@ -97,12 +93,11 @@ public class GridNearTxRemote extends GridDistributedTxRemoteAdapter {
         long timeout,
         Collection<IgniteTxEntry> writeEntries,
         int txSize,
-        @Nullable IgniteTxKey grpLockKey,
         @Nullable UUID subjId,
         int taskNameHash
     ) throws IgniteCheckedException {
         super(ctx, nodeId, rmtThreadId, xidVer, commitVer, sys, plc, concurrency, isolation, invalidate, timeout,
-            txSize, grpLockKey, subjId, taskNameHash);
+            txSize, subjId, taskNameHash);
 
         assert nearNodeId != null;
 
@@ -138,7 +133,6 @@ public class GridNearTxRemote extends GridDistributedTxRemoteAdapter {
      * @param timeout Timeout.
      * @param ctx Cache registry.
      * @param txSize Expected transaction size.
-     * @param grpLockKey Collection of group lock keys if this is a group-lock transaction.
      */
     public GridNearTxRemote(
         GridCacheSharedContext ctx,
@@ -155,12 +149,11 @@ public class GridNearTxRemote 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;
 
@@ -192,19 +185,6 @@ public class GridNearTxRemote extends GridDistributedTxRemoteAdapter {
     }
 
     /**
-     * Marks near local transaction as group lock. Note that near remote transaction may be
-     * marked as group lock even if it does not contain any locked key.
-     */
-    public void markGroupLock() {
-        grpLock = true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean groupLock() {
-        return grpLock || super.groupLock();
-    }
-
-    /**
      * @return Near transaction ID.
      */
     @Override public GridCacheVersion nearXidVersion() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
index 2bed843..5f877ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
@@ -284,16 +284,6 @@ public interface IgniteInternalTx extends AutoCloseable, GridTimeoutObject {
     public boolean empty();
 
     /**
-     * @return {@code True} if transaction group-locked.
-     */
-    public boolean groupLock();
-
-    /**
-     * @return Group lock key if {@link #groupLock()} is {@code true}.
-     */
-    @Nullable public IgniteTxKey groupLockKey();
-
-    /**
      * @return {@code True} if preparing flag was set with this call.
      */
     public boolean markPreparing();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java
index 044c3d7..99907e4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java
@@ -160,9 +160,7 @@ public class IgniteTransactionsImpl<K, V> implements IgniteTransactionsEx {
             isolation,
             timeout,
             true,
-            txSize,
-            /** group lock keys */null,
-            /** partition lock */false
+            txSize
         );
 
         assert tx != null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/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 64cc77f..eb8825e 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
@@ -179,9 +179,6 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
     /** */
     protected int txSize;
 
-    /** Group lock key, if any. */
-    protected IgniteTxKey grpLockKey;
-
     /** */
     @GridToStringExclude
     private AtomicReference<GridFutureAdapter<IgniteInternalTx>> finFut = new AtomicReference<>();
@@ -233,7 +230,6 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
      * @param isolation Isolation.
      * @param timeout Timeout.
      * @param txSize Transaction size.
-     * @param grpLockKey Group lock key if this is group-lock transaction.
      */
     protected IgniteTxAdapter(
         GridCacheSharedContext<?, ?> cctx,
@@ -249,7 +245,6 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
         boolean invalidate,
         boolean storeEnabled,
         int txSize,
-        @Nullable IgniteTxKey grpLockKey,
         @Nullable UUID subjId,
         int taskNameHash
     ) {
@@ -269,7 +264,6 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
         this.invalidate = invalidate;
         this.storeEnabled = storeEnabled;
         this.txSize = txSize;
-        this.grpLockKey = grpLockKey;
         this.subjId = subjId;
         this.taskNameHash = taskNameHash;
 
@@ -294,7 +288,6 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
      * @param isolation Isolation.
      * @param timeout Timeout.
      * @param txSize Transaction size.
-     * @param grpLockKey Group lock key if this is group-lock transaction.
      */
     protected IgniteTxAdapter(
         GridCacheSharedContext<?, ?> cctx,
@@ -308,7 +301,6 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
         TransactionIsolation isolation,
         long timeout,
         int txSize,
-        @Nullable IgniteTxKey grpLockKey,
         @Nullable UUID subjId,
         int taskNameHash
     ) {
@@ -323,7 +315,6 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
         this.isolation = isolation;
         this.timeout = timeout;
         this.txSize = txSize;
-        this.grpLockKey = grpLockKey;
         this.subjId = subjId;
         this.taskNameHash = taskNameHash;
 
@@ -387,30 +378,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
 
     /** {@inheritDoc} */
     @Override public Collection<IgniteTxEntry> optimisticLockEntries() {
-        if (!groupLock())
-            return writeEntries();
-        else {
-            if (!F.isEmpty(invalidParts)) {
-                assert invalidParts.size() == 1 : "Only one partition expected for group lock transaction " +
-                    "[tx=" + this + ", invalidParts=" + invalidParts + ']';
-                assert groupLockEntry() == null : "Group lock key should be rejected " +
-                    "[tx=" + this + ", groupLockEntry=" + groupLockEntry() + ']';
-                assert F.isEmpty(writeMap()) : "All entries should be rejected for group lock transaction " +
-                    "[tx=" + this + ", writes=" + writeMap() + ']';
-
-                return Collections.emptyList();
-            }
-
-            IgniteTxEntry grpLockEntry = groupLockEntry();
-
-            assert grpLockEntry != null || (near() && !local()):
-                "Group lock entry was not enlisted into transaction [tx=" + this +
-                ", grpLockKey=" + groupLockKey() + ']';
-
-            return grpLockEntry == null ?
-                Collections.<IgniteTxEntry>emptyList() :
-                Collections.singletonList(grpLockEntry);
-        }
+        return writeEntries();
     }
 
     /** {@inheritDoc} */
@@ -482,16 +450,6 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
         cctx.tm().uncommitTx(this);
     }
 
-    /**
-     * This method uses unchecked assignment to cast group lock key entry to transaction generic signature.
-     *
-     * @return Group lock tx entry.
-     */
-    @SuppressWarnings("unchecked")
-    public IgniteTxEntry groupLockEntry() {
-        return this.entry(groupLockKey());
-    }
-
     /** {@inheritDoc} */
     @Override public UUID otherNodeId() {
         return null;
@@ -603,16 +561,6 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
     public abstract boolean isStarted();
 
     /** {@inheritDoc} */
-    @Override public boolean groupLock() {
-        return grpLockKey != null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteTxKey groupLockKey() {
-        return grpLockKey;
-    }
-
-    /** {@inheritDoc} */
     @Override public int size() {
         return txSize;
     }
@@ -798,9 +746,6 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
 
         GridCacheVersion explicit = txEntry == null ? null : txEntry.explicitVersion();
 
-        assert !txEntry.groupLockEntry() || groupLock() : "Can not have group-locked tx entries in " +
-            "non-group-lock transactions [txEntry=" + txEntry + ", tx=" + this + ']';
-
         return local() && !cacheCtx.isDht() ?
             entry.lockedByThread(threadId()) || (explicit != null && entry.lockedBy(explicit)) :
             // If candidate is not there, then lock was explicit.
@@ -817,9 +762,6 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
 
         GridCacheVersion explicit = txEntry == null ? null : txEntry.explicitVersion();
 
-        assert !txEntry.groupLockEntry() || groupLock() : "Can not have group-locked tx entries in " +
-            "non-group-lock transactions [txEntry=" + txEntry + ", tx=" + this + ']';
-
         return local() && !cacheCtx.isDht() ?
             entry.lockedByThreadUnsafe(threadId()) || (explicit != null && entry.lockedByUnsafe(explicit)) :
             // If candidate is not there, then lock was explicit.
@@ -1554,7 +1496,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
     /** {@inheritDoc} */
     @Override public String toString() {
         return GridToStringBuilder.toString(IgniteTxAdapter.class, this,
-            "duration", (U.currentTimeMillis() - startTime) + "ms", "grpLock", groupLock(),
+            "duration", (U.currentTimeMillis() - startTime) + "ms",
             "onePhaseCommit", onePhaseCommit);
     }
 
@@ -1779,16 +1721,6 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
         }
 
         /** {@inheritDoc} */
-        @Override public boolean groupLock() {
-            return false;
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public IgniteTxKey groupLockKey() {
-            throw new IllegalStateException("Deserialized transaction can only be used as read-only.");
-        }
-
-        /** {@inheritDoc} */
         @Override public boolean markPreparing() {
             throw new IllegalStateException("Deserialized transaction can only be used as read-only.");
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/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 0d7aeaf..247d350 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
@@ -137,9 +137,6 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
     @GridDirectTransient
     private boolean locMapped;
 
-    /** Group lock entry flag. */
-    private boolean grpLock;
-
     /** Expiry policy. */
     @GridDirectTransient
     private ExpiryPolicy expiryPlc;
@@ -277,22 +274,6 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
     }
 
     /**
-     * @return {@code True} if this entry was added in group lock transaction and
-     *      this is not a group lock entry.
-     */
-    public boolean groupLockEntry() {
-        return grpLock;
-    }
-
-    /**
-     * @param grpLock {@code True} if this entry was added in group lock transaction and
-     *      this is not a group lock entry.
-     */
-    public void groupLockEntry(boolean grpLock) {
-        this.grpLock = grpLock;
-    }
-
-    /**
      * @param ctx Context.
      * @return Clean copy of this entry.
      */
@@ -311,7 +292,6 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
         cp.ttl = ttl;
         cp.conflictExpireTime = conflictExpireTime;
         cp.explicitVer = explicitVer;
-        cp.grpLock = grpLock;
         cp.conflictVer = conflictVer;
         cp.expiryPlc = expiryPlc;
         cp.flags = flags;
@@ -851,30 +831,24 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
                 writer.incrementState();
 
             case 7:
-                if (!writer.writeBoolean("grpLock", grpLock))
-                    return false;
-
-                writer.incrementState();
-
-            case 8:
                 if (!writer.writeMessage("key", key))
                     return false;
 
                 writer.incrementState();
 
-            case 9:
+            case 8:
                 if (!writer.writeByteArray("transformClosBytes", transformClosBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 10:
+            case 9:
                 if (!writer.writeLong("ttl", ttl))
                     return false;
 
                 writer.incrementState();
 
-            case 11:
+            case 10:
                 if (!writer.writeMessage("val", val))
                     return false;
 
@@ -950,14 +924,6 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
                 reader.incrementState();
 
             case 7:
-                grpLock = reader.readBoolean("grpLock");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 8:
                 key = reader.readMessage("key");
 
                 if (!reader.isLastRead())
@@ -965,7 +931,7 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
 
                 reader.incrementState();
 
-            case 9:
+            case 8:
                 transformClosBytes = reader.readByteArray("transformClosBytes");
 
                 if (!reader.isLastRead())
@@ -973,7 +939,7 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
 
                 reader.incrementState();
 
-            case 10:
+            case 9:
                 ttl = reader.readLong("ttl");
 
                 if (!reader.isLastRead())
@@ -981,7 +947,7 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
 
                 reader.incrementState();
 
-            case 11:
+            case 10:
                 val = reader.readMessage("val");
 
                 if (!reader.isLastRead())
@@ -1001,7 +967,7 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 12;
+        return 11;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index 826f392..f466bf2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -271,8 +271,6 @@ public class IgniteTxHandler {
                 req.isInvalidate(),
                 false,
                 req.txSize(),
-                req.groupLockKey(),
-                req.partitionLock(),
                 req.transactionNodes(),
                 req.subjectId(),
                 req.taskNameHash()
@@ -554,8 +552,6 @@ public class IgniteTxHandler {
                             req.isInvalidate(),
                             req.storeEnabled(),
                             req.txSize(),
-                            req.groupLockKey(),
-                            false,
                             null,
                             req.subjectId(),
                             req.taskNameHash()));
@@ -1002,7 +998,6 @@ public class IgniteTxHandler {
                     req.isInvalidate(),
                     req.timeout(),
                     req.writes() != null ? Math.max(req.writes().size(), req.txSize()) : req.txSize(),
-                    req.groupLockKey(),
                     req.nearXidVersion(),
                     req.transactionNodes(),
                     req.subjectId(),
@@ -1136,7 +1131,6 @@ public class IgniteTxHandler {
                     req.timeout(),
                     req.nearWrites(),
                     req.txSize(),
-                    req.groupLockKey(),
                     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/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 7e9095c..609108f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -86,9 +86,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
     /** Base for completed versions. */
     private GridCacheVersion completedBase;
 
-    /** Flag indicating partition lock in group lock transaction. */
-    private boolean partLock;
-
     /** Flag indicating that transformed values should be sent to remote nodes. */
     private boolean sndTransformedVals;
 
@@ -123,8 +120,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
      * @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 {@code True} if this is a group-lock transaction and lock is acquired for whole partition.
      */
     protected IgniteTxLocalAdapter(
         GridCacheSharedContext cctx,
@@ -139,17 +134,11 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
         boolean invalidate,
         boolean storeEnabled,
         int txSize,
-        @Nullable IgniteTxKey grpLockKey,
-        boolean partLock,
         @Nullable UUID subjId,
         int taskNameHash
     ) {
         super(cctx, xidVer, implicit, implicitSingle, /*local*/true, sys, plc, concurrency, isolation, timeout,
-            invalidate, storeEnabled, txSize, grpLockKey, subjId, taskNameHash);
-
-        assert !partLock || grpLockKey != null;
-
-        this.partLock = partLock;
+            invalidate, storeEnabled, txSize, subjId, taskNameHash);
 
         minVer = xidVer;
     }
@@ -182,11 +171,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
     }
 
     /** {@inheritDoc} */
-    @Override public boolean partitionLock() {
-        return partLock;
-    }
-
-    /** {@inheritDoc} */
     @Override public Throwable commitError() {
         return commitErr.get();
     }
@@ -499,7 +483,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
         CacheStoreManager store = store();
 
         if (store != null && store.isWriteThrough() && storeEnabled() &&
-            (!internal() || groupLock()) && (near() || store.isWriteToStoreFromDht())) {
+            !internal() && (near() || store.isWriteToStoreFromDht())) {
             try {
                 if (writeEntries != null) {
                     Map<Object, IgniteBiTuple<Object, GridCacheVersion>> putMap = null;
@@ -679,9 +663,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
         if (!empty || colocated())
             cctx.tm().addCommittedTx(this);
 
-        if (groupLock())
-            addGroupTxMapping(writeSet());
-
         if (!empty) {
             batchStoreCommit(writeMap().values());
 
@@ -909,10 +890,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                             log.debug("Ignoring READ entry when committing: " + txEntry);
                                     }
                                     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());
@@ -927,7 +904,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                 // we are not changing obsolete entries.
                                 // (innerSet and innerRemove will throw an exception
                                 // if an entry is obsolete).
-                                if (txEntry.op() != READ && !txEntry.groupLockEntry())
+                                if (txEntry.op() != READ)
                                     checkCommitLocks(cached);
 
                                 // Break out of while loop.
@@ -996,7 +973,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
         else {
             CacheStoreManager store = store();
 
-            if (store != null && (!internal() || groupLock())) {
+            if (store != null && !internal()) {
                 try {
                     store.sessionEnd(this, true);
                 }
@@ -1102,7 +1079,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                 CacheStoreManager store = store();
 
                 if (store != null && (near() || store.isWriteToStoreFromDht())) {
-                    if (!internal() || groupLock())
+                    if (!internal())
                         store.sessionEnd(this, false);
                 }
             }
@@ -1152,8 +1129,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
 
         cacheCtx.checkSecurity(SecurityPermission.CACHE_READ);
 
-        groupLockSanityCheck(cacheCtx, keys);
-
         boolean single = keysCnt == 1;
 
         Collection<KeyCacheObject> lockKeys = null;
@@ -1185,7 +1160,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                         cacheCtx.addResult(map, key, val, skipVals, keepCacheObjects, deserializePortable, false);
                 }
                 else {
-                    assert txEntry.op() == TRANSFORM || (groupLock() && !txEntry.groupLockEntry());
+                    assert txEntry.op() == TRANSFORM;
 
                     while (true) {
                         try {
@@ -1263,7 +1238,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
 
                         CacheObject val = null;
 
-                        if (!pessimistic() || readCommitted() || groupLock() && !skipVals) {
+                        if (!pessimistic() || readCommitted() && !skipVals) {
                             IgniteCacheExpiryPolicy accessPlc =
                                 optimistic() ? accessPolicy(cacheCtx, txKey, expiryPlc) : null;
 
@@ -1311,8 +1286,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                 null,
                                 skipStore);
 
-                            if (groupLock())
-                                txEntry.groupLockEntry(true);
 
                             // As optimization, mark as checked immediately
                             // for non-pessimistic if value is not null.
@@ -1527,7 +1500,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                             nextVer = cctx.versions().next(topologyVersion());
 
                         while (true) {
-                            assert txEntry != null || readCommitted() || groupLock() || skipVals;
+                            assert txEntry != null || readCommitted() || skipVals;
 
                             GridCacheEntryEx e = txEntry == null ? entryEx(cacheCtx, txKey) : txEntry.cached();
 
@@ -1544,8 +1517,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                         log.debug("Got removed entry in transaction getAll method " +
                                             "(will try again): " + e);
 
-                                    if (pessimistic() && !readCommitted() && !isRollbackOnly() &&
-                                        (!groupLock() || F.eq(e.key(), groupLockKey()))) {
+                                    if (pessimistic() && !readCommitted() && !isRollbackOnly()) {
                                         U.error(log, "Inconsistent transaction state (entry got removed while " +
                                             "holding lock) [entry=" + e + ", tx=" + IgniteTxLocalAdapter.this + "]");
 
@@ -1563,7 +1535,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                 // In pessimistic mode, we should always be able to set.
                                 assert set || !pessimistic();
 
-                                if (readCommitted() || groupLock() || skipVals) {
+                                if (readCommitted() || skipVals) {
                                     cacheCtx.evicts().touch(e, topologyVersion());
 
                                     if (visibleVal != null) {
@@ -1654,7 +1626,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                 return new GridFinishedFuture<>(retMap);
 
             // Handle locks.
-            if (pessimistic() && !readCommitted() && !groupLock() && !skipVals) {
+            if (pessimistic() && !readCommitted() && !skipVals) {
                 if (expiryPlc == null)
                     expiryPlc = cacheCtx.expiry();
 
@@ -1811,7 +1783,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                 }
             }
             else {
-                assert optimistic() || readCommitted() || groupLock() || skipVals;
+                assert optimistic() || readCommitted() || skipVals;
 
                 final Collection<KeyCacheObject> redos = new ArrayList<>();
 
@@ -2036,8 +2008,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
             if (invokeMap != null)
                 transform = true;
 
-            groupLockSanityCheck(cacheCtx, keys);
-
             for (Object key : keys) {
                 if (key == null) {
                     rollback();
@@ -2194,12 +2164,9 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                             if (!implicit() && readCommitted() && !cacheCtx.offheapTiered())
                                 cacheCtx.evicts().touch(entry, topologyVersion());
 
-                            if (groupLock() && !lockOnly)
-                                txEntry.groupLockEntry(true);
-
                             enlisted.add(cacheKey);
 
-                            if ((!pessimistic() && !implicit()) || (groupLock() && !lockOnly)) {
+                            if (!pessimistic() && !implicit()) {
                                 txEntry.markValid();
 
                                 if (old == null) {
@@ -2644,7 +2611,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                 null,
                 opCtx != null && opCtx.skipStore());
 
-            if (pessimistic() && !groupLock()) {
+            if (pessimistic()) {
                 // Loose all skipped.
                 final Set<KeyCacheObject> loaded = loadFut.get();
 
@@ -2867,7 +2834,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
             // Acquire locks only after having added operation to the write set.
             // Otherwise, during rollback we will not know whether locks need
             // to be rolled back.
-            if (pessimistic() && !groupLock()) {
+            if (pessimistic()) {
                 // Loose all skipped.
                 final Collection<KeyCacheObject> passedKeys = F.view(enlisted, F0.notIn(loadFut.get()));
 
@@ -2985,108 +2952,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
     }
 
     /**
-     * Adds key mapping to transaction.
-     * @param keys Keys to add.
-     */
-    protected void addGroupTxMapping(Collection<IgniteTxKey> keys) {
-        // No-op. This method is overriden in transactions that store key to remote node mapping
-        // for commit.
-    }
-
-    /**
-     * Checks that affinity keys are enlisted in group transaction on start.
-     *
-     * @param cacheCtx Cache context.
-     * @param keys Keys to check.
-     * @throws IgniteCheckedException If sanity check failed.
-     */
-    private <K> void groupLockSanityCheck(GridCacheContext cacheCtx, Iterable<? extends K> keys)
-        throws IgniteCheckedException
-    {
-        if (groupLock() && cctx.kernalContext().config().isCacheSanityCheckEnabled()) {
-            // Note that affinity is called without mapper on purpose.
-            int affinityPart = cacheCtx.config().getAffinity().partition(grpLockKey.key());
-
-            for (K key : keys) {
-                if (partitionLock()) {
-                    int part = cacheCtx.affinity().partition(key);
-
-                    if (affinityPart != part)
-                        throw new IgniteCheckedException("Failed to enlist key into group-lock transaction (given " +
-                            "key does not belong to locked partition) [key=" + key + ", affinityPart=" + affinityPart +
-                            ", part=" + part + ", groupLockKey=" + grpLockKey + ']');
-                }
-                else {
-                    KeyCacheObject cacheKey =
-                        cacheCtx.toCacheKeyObject(cacheCtx.config().getAffinityMapper().affinityKey(key));
-
-                    IgniteTxKey affinityKey = cacheCtx.txKey(cacheKey);
-
-                    if (!grpLockKey.equals(affinityKey))
-                        throw new IgniteCheckedException("Failed to enlist key into group-lock transaction (affinity key was " +
-                            "not enlisted to transaction on start) [key=" + key + ", affinityKey=" + affinityKey +
-                            ", groupLockKey=" + grpLockKey + ']');
-                }
-            }
-        }
-    }
-
-    /**
-     * Performs keys locking for affinity-based group lock transactions.
-     * @return Lock future.
-     */
-    @Override public <K> IgniteInternalFuture<?> groupLockAsync(GridCacheContext cacheCtx, Collection<K> keys) {
-        assert groupLock();
-
-        try {
-            init();
-
-            GridCacheReturn ret = new GridCacheReturn(localResult(), false);
-
-            Collection<KeyCacheObject> enlisted = new ArrayList<>();
-
-            Set<KeyCacheObject> skipped = enlistWrite(
-                cacheCtx,
-                keys,
-                /** cached entry */null,
-                /** expiry - leave unchanged */null,
-                /** implicit */false,
-                /** lookup map */null,
-                /** invoke map */null,
-                /** invoke arguments */null,
-                /** retval */false,
-                /** lock only */true,
-                CU.empty0(),
-                ret,
-                enlisted,
-                null,
-                null,
-                cacheCtx.skipStore()
-            ).get();
-
-            // No keys should be skipped with empty filter.
-            assert F.isEmpty(skipped);
-
-            // Lock group key in pessimistic mode only.
-            return pessimistic() ?
-                cacheCtx.cache().txLockAsync(enlisted,
-                    lockTimeout(),
-                    this,
-                    false,
-                    false,
-                    isolation,
-                    isInvalidate(),
-                    -1L) :
-                new GridFinishedFuture<>();
-        }
-        catch (IgniteCheckedException e) {
-            setRollbackOnly();
-
-            return new GridFinishedFuture<Object>(e);
-        }
-    }
-
-    /**
      * Initializes read map.
      *
      * @return {@code True} if transaction was successfully  started.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java
index 61041e1..14562ab 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java
@@ -58,14 +58,9 @@ public interface IgniteTxLocalEx extends IgniteInternalTx {
     public void userRollback() throws IgniteCheckedException;
 
     /**
-     * @return Group lock entry if this is a group-lock transaction.
-     */
-    @Nullable public IgniteTxEntry groupLockEntry();
-
-    /**
      * @param cacheCtx Cache context.
      * @param keys Keys to get.
-     * @param cached Cached entry if this method is called from entry wrapper.
+     * @param cached Cached entry if this method is called from entry wrapper
      *      Cached entry is passed if and only if there is only one key in collection of keys.
      * @param deserializePortable Deserialize portable flag.
      * @param skipVals Skip values flag.
@@ -144,20 +139,6 @@ public interface IgniteTxLocalEx extends IgniteInternalTx {
         Map<KeyCacheObject, GridCacheVersion> drMap);
 
     /**
-     * Performs keys locking for affinity-based group lock transactions.
-     *
-     * @param cacheCtx Cache context.
-     * @param keys Keys to lock.
-     * @return Lock future.
-     */
-    public <K> IgniteInternalFuture<?> groupLockAsync(GridCacheContext cacheCtx, Collection<K> keys);
-
-    /**
-     * @return {@code True} if keys from the same partition are allowed to be enlisted in group-lock transaction.
-     */
-    public boolean partitionLock();
-
-    /**
      * @return Return value for
      */
     public GridCacheReturn implicitSingleResult();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
index 2122602..8e95a5d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
@@ -347,8 +347,6 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
      * @param isolation Isolation.
      * @param timeout transaction timeout.
      * @param txSize Expected transaction size.
-     * @param grpLockKey Group lock key if this is a group-lock transaction.
-     * @param partLock {@code True} if partition is locked.
      * @return New transaction.
      */
     public IgniteTxLocalAdapter newTx(
@@ -359,9 +357,7 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
         TransactionIsolation isolation,
         long timeout,
         boolean storeEnabled,
-        int txSize,
-        @Nullable IgniteTxKey grpLockKey,
-        boolean partLock) {
+        int txSize) {
         assert sysCacheCtx == null || sysCacheCtx.systemTx();
 
         UUID subjId = null; // TODO GG-9141 how to get subj ID?
@@ -379,8 +375,6 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
             timeout,
             storeEnabled,
             txSize,
-            grpLockKey,
-            partLock,
             subjId,
             taskNameHash);
 
@@ -1207,13 +1201,10 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
             cctx.kernalContext().dataStructures().onTxCommitted(tx);
 
             // 4. Unlock write resources.
-            if (tx.groupLock())
-                unlockGroupLocks(tx);
-            else
-                unlockMultiple(tx, tx.writeEntries());
+            unlockMultiple(tx, tx.writeEntries());
 
             // 5. For pessimistic transaction, unlock read resources if required.
-            if (tx.pessimistic() && !tx.readCommitted() && !tx.groupLock())
+            if (tx.pessimistic() && !tx.readCommitted())
                 unlockMultiple(tx, tx.readEntries());
 
             // 6. Notify evictions.
@@ -1441,7 +1432,7 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
      * @param tx Transaction to notify evictions for.
      */
     private void notifyEvitions(IgniteInternalTx tx) {
-        if (tx.internal() && !tx.groupLock())
+        if (tx.internal())
             return;
 
         for (IgniteTxEntry txEntry : tx.allEntries())
@@ -1617,51 +1608,6 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
-     * Unlocks entries locked by group transaction.
-     *
-     * @param txx Transaction.
-     */
-    @SuppressWarnings("unchecked")
-    private void unlockGroupLocks(IgniteInternalTx txx) {
-        IgniteTxKey grpLockKey = txx.groupLockKey();
-
-        assert grpLockKey != null;
-
-        if (grpLockKey == null)
-            return;
-
-        IgniteTxEntry txEntry = txx.entry(grpLockKey);
-
-        assert txEntry != null || (txx.near() && !txx.local());
-
-        if (txEntry != null) {
-            GridCacheContext cacheCtx = txEntry.context();
-
-            // Group-locked entries must be locked.
-            while (true) {
-                try {
-                    GridCacheEntryEx entry = txEntry.cached();
-
-                    assert entry != null;
-
-                    entry.txUnlock(txx);
-
-                    break;
-                }
-                catch (GridCacheEntryRemovedException ignored) {
-                    if (log.isDebugEnabled())
-                        log.debug("Got removed entry in TM unlockGroupLocks(..) method (will retry): " + txEntry);
-
-                    GridCacheAdapter cache = cacheCtx.cache();
-
-                    // Renew cache entry.
-                    txEntry.cached(cache.entryEx(txEntry.key()));
-                }
-            }
-        }
-    }
-
-    /**
      * @param tx Owning transaction.
      * @param entries Entries to unlock.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da5a2282/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
index 56bd676..9af29d6 100644
--- a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
+++ b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
@@ -85,9 +85,7 @@ public class CacheJtaManager extends CacheJtaManagerAdapter {
                                 tCfg.getDefaultTxIsolation(),
                                 tCfg.getDefaultTxTimeout(),
                                 /*store enabled*/true,
-                                /*tx size*/0,
-                                /*group lock keys*/null,
-                                /*partition lock*/false
+                                /*tx size*/0
                             );
                         }