You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by dm...@apache.org on 2017/03/25 00:12:46 UTC

[06/56] [abbrv] ignite git commit: Internal cache API cleanup.

http://git-wip-us.apache.org/repos/asf/ignite/blob/decb0c7a/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 229961e..48b508b 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
@@ -46,6 +46,15 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** */
+    private static final int NEED_RETURN_VALUE_FLAG_MASK = 0x01;
+
+    /** */
+    private static final int FIRST_CLIENT_REQ_FLAG_MASK = 0x02;
+
+    /** */
+    private static final int SYNC_COMMIT_FLAG_MASK = 0x04;
+
     /** Topology version. */
     private AffinityTopologyVersion topVer;
 
@@ -55,15 +64,6 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
     /** Filter. */
     private CacheEntryPredicate[] filter;
 
-    /** Implicit flag. */
-    private boolean implicitTx;
-
-    /** Implicit transaction with one key flag. */
-    private boolean implicitSingleTx;
-
-    /** Flag is kept for backward compatibility. */
-    private boolean onePhaseCommit;
-
     /** Array of mapped DHT versions for this entry. */
     @GridToStringInclude
     private GridCacheVersion[] dhtVers;
@@ -74,23 +74,14 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
     /** Task name hash. */
     private int taskNameHash;
 
-    /** Has transforms flag. */
-    private boolean hasTransforms;
-
-    /** Sync commit flag. */
-    private boolean syncCommit;
-
     /** TTL for create operation. */
     private long createTtl;
 
     /** TTL for read operation. */
     private long accessTtl;
 
-    /** Flag indicating whether cache operation requires a previous value. */
-    private boolean retVal;
-
-    /** {@code True} if first lock request for lock operation sent from client node. */
-    private boolean firstClientReq;
+    /** */
+    private byte flags;
 
     /**
      * Empty constructor required for {@link Externalizable}.
@@ -107,8 +98,6 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
      * @param futId Future ID.
      * @param lockVer Cache version.
      * @param isInTx {@code True} if implicit transaction lock.
-     * @param implicitTx Flag to indicate that transaction is implicit.
-     * @param implicitSingleTx Implicit-transaction-with-one-key flag.
      * @param isRead Indicates whether implicit lock is for read or write operation.
      * @param retVal Return value flag.
      * @param isolation Transaction isolation.
@@ -133,8 +122,6 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
         IgniteUuid futId,
         GridCacheVersion lockVer,
         boolean isInTx,
-        boolean implicitTx,
-        boolean implicitSingleTx,
         boolean isRead,
         boolean retVal,
         TransactionIsolation isolation,
@@ -174,24 +161,43 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
         assert topVer.compareTo(AffinityTopologyVersion.ZERO) > 0;
 
         this.topVer = topVer;
-        this.implicitTx = implicitTx;
-        this.implicitSingleTx = implicitSingleTx;
-        this.syncCommit = syncCommit;
         this.subjId = subjId;
         this.taskNameHash = taskNameHash;
         this.createTtl = createTtl;
         this.accessTtl = accessTtl;
-        this.retVal = retVal;
-        this.firstClientReq = firstClientReq;
 
         dhtVers = new GridCacheVersion[keyCnt];
+
+        setFlag(syncCommit, SYNC_COMMIT_FLAG_MASK);
+        setFlag(firstClientReq, FIRST_CLIENT_REQ_FLAG_MASK);
+        setFlag(retVal, NEED_RETURN_VALUE_FLAG_MASK);
+    }
+
+    /**
+     * Sets flag mask.
+     *
+     * @param flag Set or clear.
+     * @param mask Mask.
+     */
+    private void setFlag(boolean flag, int mask) {
+        flags = flag ? (byte)(flags | mask) : (byte)(flags & ~mask);
+    }
+
+    /**
+     * Reags flag mask.
+     *
+     * @param mask Mask to read.
+     * @return Flag value.
+     */
+    private boolean isFlag(int mask) {
+        return (flags & mask) != 0;
     }
 
     /**
      * @return {@code True} if first lock request for lock operation sent from client node.
      */
     public boolean firstClientRequest() {
-        return firstClientReq;
+        return isFlag(FIRST_CLIENT_REQ_FLAG_MASK);
     }
 
     /**
@@ -216,24 +222,10 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
     }
 
     /**
-     * @return Implicit transaction flag.
-     */
-    public boolean implicitTx() {
-        return implicitTx;
-    }
-
-    /**
-     * @return Implicit-transaction-with-one-key flag.
-     */
-    public boolean implicitSingleTx() {
-        return implicitSingleTx;
-    }
-
-    /**
      * @return Sync commit flag.
      */
     public boolean syncCommit() {
-        return syncCommit;
+        return isFlag(SYNC_COMMIT_FLAG_MASK);
     }
 
     /**
@@ -268,24 +260,10 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
     }
 
     /**
-     * @param hasTransforms {@code True} if originating transaction has transform entries.
-     */
-    public void hasTransforms(boolean hasTransforms) {
-        this.hasTransforms = hasTransforms;
-    }
-
-    /**
-     * @return {@code True} if originating transaction has transform entries.
-     */
-    public boolean hasTransforms() {
-        return hasTransforms;
-    }
-
-    /**
      * @return Need return value flag.
      */
     public boolean needReturnValue() {
-        return retVal;
+        return isFlag(NEED_RETURN_VALUE_FLAG_MASK);
     }
 
     /**
@@ -399,66 +377,30 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
                 writer.incrementState();
 
             case 24:
-                if (!writer.writeBoolean("firstClientReq", firstClientReq))
+                if (!writer.writeByte("flags", flags))
                     return false;
 
                 writer.incrementState();
 
             case 25:
-                if (!writer.writeBoolean("hasTransforms", hasTransforms))
-                    return false;
-
-                writer.incrementState();
-
-            case 26:
-                if (!writer.writeBoolean("implicitSingleTx", implicitSingleTx))
-                    return false;
-
-                writer.incrementState();
-
-            case 27:
-                if (!writer.writeBoolean("implicitTx", implicitTx))
-                    return false;
-
-                writer.incrementState();
-
-            case 28:
                 if (!writer.writeInt("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
-            case 29:
-                if (!writer.writeBoolean("onePhaseCommit", onePhaseCommit))
-                    return false;
-
-                writer.incrementState();
-
-            case 30:
-                if (!writer.writeBoolean("retVal", retVal))
-                    return false;
-
-                writer.incrementState();
-
-            case 31:
+            case 26:
                 if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
-            case 32:
-                if (!writer.writeBoolean("syncCommit", syncCommit))
-                    return false;
-
-                writer.incrementState();
-
-            case 33:
+            case 27:
                 if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
-            case 34:
+            case 28:
                 if (!writer.writeMessage("topVer", topVer))
                     return false;
 
@@ -513,7 +455,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
                 reader.incrementState();
 
             case 24:
-                firstClientReq = reader.readBoolean("firstClientReq");
+                flags = reader.readByte("flags");
 
                 if (!reader.isLastRead())
                     return false;
@@ -521,30 +463,6 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
                 reader.incrementState();
 
             case 25:
-                hasTransforms = reader.readBoolean("hasTransforms");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 26:
-                implicitSingleTx = reader.readBoolean("implicitSingleTx");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 27:
-                implicitTx = reader.readBoolean("implicitTx");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 28:
                 miniId = reader.readInt("miniId");
 
                 if (!reader.isLastRead())
@@ -552,23 +470,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 29:
-                onePhaseCommit = reader.readBoolean("onePhaseCommit");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 30:
-                retVal = reader.readBoolean("retVal");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 31:
+            case 26:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -576,15 +478,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 32:
-                syncCommit = reader.readBoolean("syncCommit");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 33:
+            case 27:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -592,7 +486,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
                 reader.incrementState();
 
-            case 34:
+            case 28:
                 topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
@@ -612,7 +506,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 35;
+        return 29;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/decb0c7a/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 940dd80..5ad05b0 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
@@ -133,7 +133,7 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
         if (keyCheck)
             validateCacheKeys(keys);
 
-        IgniteTxLocalAdapter tx = ctx.tm().threadLocalTx(ctx);
+        GridNearTxLocal tx = ctx.tm().threadLocalTx(ctx);
 
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
@@ -141,7 +141,7 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
 
         if (tx != null && !tx.implicit() && !skipTx) {
             return asyncOp(tx, new AsyncOp<Map<K, V>>(keys) {
-                @Override public IgniteInternalFuture<Map<K, V>> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
+                @Override public IgniteInternalFuture<Map<K, V>> op(GridNearTxLocal tx, AffinityTopologyVersion readyTopVer) {
                     return tx.getAllAsync(ctx,
                         readyTopVer,
                         ctx.cacheKeysView(keys),
@@ -377,7 +377,7 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
                                 log.debug("Node requesting lock left grid (lock request will be ignored): " + req);
 
                             if (tx != null)
-                                tx.rollback();
+                                tx.rollbackRemoteTx();
 
                             return null;
                         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/decb0c7a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
index 7387501..1b0566b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
@@ -310,7 +310,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
                         err = new TransactionRollbackException("Failed to commit transaction.", err);
 
                     try {
-                        tx.finish(err == null);
+                        tx.localFinish(err == null);
                     }
                     catch (IgniteCheckedException e) {
                         if (err != null)
@@ -402,7 +402,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
         }
 
         try {
-            if (tx.finish(commit) || (!commit && tx.state() == UNKNOWN)) {
+            if (tx.localFinish(commit) || (!commit && tx.state() == UNKNOWN)) {
                 if ((tx.onePhaseCommit() && needFinishOnePhase(commit)) || (!tx.onePhaseCommit() && mappings != null)) {
                     if (mappings.single()) {
                         GridDistributedTxMapping mapping = mappings.singleMapping();