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/15 12:37:57 UTC
[3/7] incubator-ignite git commit: #ignite-797: remove grpLockKey and
partLock from GridDistributedTxPrepareRequest.
#ignite-797: remove grpLockKey and partLock from GridDistributedTxPrepareRequest.
Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/cdda7fd3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/cdda7fd3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/cdda7fd3
Branch: refs/heads/ignite-797
Commit: cdda7fd307634dcead61ab336e58693a6d08cf3b
Parents: 00c2afd
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri May 15 13:00:52 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri May 15 13:00:52 2015 +0300
----------------------------------------------------------------------
.../GridDistributedTxPrepareRequest.java | 112 ++++---------------
.../distributed/dht/GridDhtTxPrepareFuture.java | 4 -
.../dht/GridDhtTxPrepareRequest.java | 60 +++++-----
.../near/GridNearOptimisticTxPrepareFuture.java | 2 -
.../GridNearPessimisticTxPrepareFuture.java | 2 -
.../near/GridNearTxPrepareRequest.java | 52 ++++-----
.../cache/transactions/IgniteTxHandler.java | 8 +-
7 files changed, 79 insertions(+), 161 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cdda7fd3/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/cdda7fd3/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..6ae2f74 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
@@ -803,8 +803,6 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
tx,
dhtWrites,
nearWrites,
- tx.groupLockKey(),
- tx.partitionLock(),
txNodes,
tx.nearXidVersion(),
true,
@@ -906,8 +904,6 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
tx,
null,
nearMapping.writes(),
- tx.groupLockKey(),
- tx.partitionLock(),
tx.transactionNodes(),
tx.nearXidVersion(),
true,
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cdda7fd3/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/cdda7fd3/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..c36b324 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(),
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cdda7fd3/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/cdda7fd3/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/cdda7fd3/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..b247896 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,8 @@ public class IgniteTxHandler {
req.isInvalidate(),
false,
req.txSize(),
- req.groupLockKey(),
- req.partitionLock(),
+ null,
+ false,
req.transactionNodes(),
req.subjectId(),
req.taskNameHash()
@@ -1002,7 +1002,7 @@ public class IgniteTxHandler {
req.isInvalidate(),
req.timeout(),
req.writes() != null ? Math.max(req.writes().size(), req.txSize()) : req.txSize(),
- req.groupLockKey(),
+ null,
req.nearXidVersion(),
req.transactionNodes(),
req.subjectId(),
@@ -1136,7 +1136,7 @@ public class IgniteTxHandler {
req.timeout(),
req.nearWrites(),
req.txSize(),
- req.groupLockKey(),
+ null,
req.subjectId(),
req.taskNameHash()
);