ignite-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From voze...@apache.org
Subject [11/20] ignite git commit: Moved near data to extras.
Date Fri, 29 Apr 2016 10:55:50 GMT
Moved near data to extras.


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

Branch: refs/heads/ignite-2523-1
Commit: 5f55c4dd1e14797779781fcee3b3faaac6a0e358
Parents: e522320
Author: vozerov-gridgain <vozerov@gridgain.com>
Authored: Wed Apr 27 10:46:36 2016 +0300
Committer: vozerov-gridgain <vozerov@gridgain.com>
Committed: Wed Apr 27 10:46:36 2016 +0300

----------------------------------------------------------------------
 .../GridNearAtomicSingleUpdateResponse.java     | 147 ++++++++++++-------
 1 file changed, 96 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5f55c4dd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateResponse.java
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateResponse.java
index b807896..212344f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateResponse.java
@@ -70,24 +70,8 @@ public class GridNearAtomicSingleUpdateResponse extends GridNearAtomicAbstractUp
     @GridToStringInclude
     private boolean remapKey;
 
-    /** Indexes of keys for which values were generated on primary node (used if originating
node has near cache). */
-    private boolean nearValsIdx;
-
-    /** Indexes of keys for which update was skipped (used if originating node has near cache).
*/
-    private boolean nearSkipIdx;
-
-    /** Values generated on primary node which should be put to originating node's near cache.
*/
-    @GridToStringInclude
-    private CacheObject nearVal;
-
-    /** Version generated on primary node to be used for originating node's near cache update.
*/
-    private GridCacheVersion nearVer;
-
-    /** Near TTLs. */
-    private long nearTtl = -1L;
-
-    /** Near expire times. */
-    private long nearExpireTime = -1L;
+    /** Near extras. */
+    private NearExtras nearExtras;
 
     /**
      * Empty constructor required by {@link Externalizable}.
@@ -186,10 +170,12 @@ public class GridNearAtomicSingleUpdateResponse extends GridNearAtomicAbstractUp
     @Override public void addNearValue(int keyIdx, @Nullable CacheObject val, long ttl, long
expireTime) {
         assert keyIdx == 0;
 
+        initializeNearExtras();
+
         addNearTtl(keyIdx, ttl, expireTime);
 
-        nearValsIdx = true;
-        nearVal = val;
+        nearExtras.valsIdx = true;
+        nearExtras.val = val;
     }
 
     /** {@inheritDoc} */
@@ -197,39 +183,45 @@ public class GridNearAtomicSingleUpdateResponse extends GridNearAtomicAbstractUp
     @Override public void addNearTtl(int keyIdx, long ttl, long expireTime) {
         assert keyIdx == 0;
 
-        nearTtl = ttl;
-        nearExpireTime = expireTime;
+        initializeNearExtras();
+
+        nearExtras.ttl = ttl;
+        nearExtras.expireTime = expireTime;
     }
 
     /** {@inheritDoc} */
     @Override public long nearExpireTime(int idx) {
         assert idx == 0;
 
-        return nearExpireTime;
+        return nearExtras != null ? nearExtras.expireTime : -1;
     }
 
     /** {@inheritDoc} */
     @Override public long nearTtl(int idx) {
         assert idx == 0;
 
-        return nearTtl;
+        return nearExtras != null ? nearExtras.ttl : -1;
     }
 
     /** {@inheritDoc} */
     @Override public void nearVersion(GridCacheVersion nearVer) {
-        this.nearVer = nearVer;
+        initializeNearExtras();
+
+        nearExtras.ver = nearVer;
     }
 
     /** {@inheritDoc} */
     @Override public GridCacheVersion nearVersion() {
-        return nearVer;
+        return nearExtras != null ? nearExtras.ver : null;
     }
 
     /** {@inheritDoc} */
     @Override public void addSkippedIndex(int keyIdx) {
         assert keyIdx == 0;
 
-        nearSkipIdx = true;
+        initializeNearExtras();
+
+        nearExtras.skipIdx = true;
 
         addNearTtl(keyIdx, -1L, -1L);
     }
@@ -238,21 +230,21 @@ public class GridNearAtomicSingleUpdateResponse extends GridNearAtomicAbstractUp
     @Override public boolean isNearSkippedIndex(int idx) {
         assert idx == 0;
 
-        return nearSkipIdx;
+        return nearExtras != null && nearExtras.skipIdx;
     }
 
     /** {@inheritDoc} */
     @Override public boolean isNearValueIndex(int idx) {
         assert idx == 0;
 
-        return nearValsIdx;
+        return nearExtras != null && nearExtras.valsIdx;
     }
 
     /** {@inheritDoc} */
     @Override @Nullable public CacheObject nearValue(int idx) {
         assert idx == 0;
 
-        return nearVal;
+        return nearExtras != null ? nearExtras.val : null;
     }
 
     /** {@inheritDoc} */
@@ -299,6 +291,14 @@ public class GridNearAtomicSingleUpdateResponse extends GridNearAtomicAbstractUp
         return true;
     }
 
+    /**
+     * Initialize near extras.
+     */
+    private void initializeNearExtras() {
+        if (nearExtras == null)
+            nearExtras = new NearExtras();
+    }
+
     /** {@inheritDoc}
      * @param ctx*/
     @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException
{
@@ -309,7 +309,8 @@ public class GridNearAtomicSingleUpdateResponse extends GridNearAtomicAbstractUp
 
         GridCacheContext cctx = ctx.cacheContext(cacheId);
 
-        prepareMarshalCacheObject(nearVal, cctx);
+        if (nearExtras != null)
+            prepareMarshalCacheObject(nearExtras.val, cctx);
 
         if (ret != null)
             ret.prepareMarshal(cctx);
@@ -324,14 +325,13 @@ public class GridNearAtomicSingleUpdateResponse extends GridNearAtomicAbstractUp
 
         GridCacheContext cctx = ctx.cacheContext(cacheId);
 
-        finishUnmarshalCacheObject(nearVal, cctx, ldr);
+        if (nearExtras != null)
+            finishUnmarshalCacheObject(nearExtras.val, cctx, ldr);
 
         if (ret != null)
             ret.finishUnmarshal(cctx, ldr);
     }
 
-
-
     /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
@@ -366,53 +366,61 @@ public class GridNearAtomicSingleUpdateResponse extends GridNearAtomicAbstractUp
                 writer.incrementState();
 
             case 6:
-                if (!writer.writeLong("nearExpireTime", nearExpireTime))
+                if (!writer.writeBoolean("remapKey", remapKey))
                     return false;
 
                 writer.incrementState();
 
             case 7:
-                if (!writer.writeBoolean("nearSkipIdx", nearSkipIdx))
+                if (!writer.writeMessage("ret", ret))
                     return false;
 
                 writer.incrementState();
 
             case 8:
-                if (!writer.writeLong("nearTtl", nearTtl))
+                if (!writer.writeBoolean("hasNearExtras", nearExtras != null))
                     return false;
 
                 writer.incrementState();
 
+                if (nearExtras == null)
+                    return true;
+
             case 9:
-                if (!writer.writeMessage("nearVal", nearVal))
+                if (!writer.writeLong("nearExpireTime", nearExtras.expireTime))
                     return false;
 
                 writer.incrementState();
 
             case 10:
-                if (!writer.writeBoolean("nearValsIdx", nearValsIdx))
+                if (!writer.writeBoolean("nearSkipIdx", nearExtras.skipIdx))
                     return false;
 
                 writer.incrementState();
 
             case 11:
-                if (!writer.writeMessage("nearVer", nearVer))
+                if (!writer.writeLong("nearTtl", nearExtras.ttl))
                     return false;
 
                 writer.incrementState();
 
             case 12:
-                if (!writer.writeBoolean("remapKey", remapKey))
+                if (!writer.writeMessage("nearVal", nearExtras.val))
                     return false;
 
                 writer.incrementState();
 
             case 13:
-                if (!writer.writeMessage("ret", ret))
+                if (!writer.writeBoolean("nearValsIdx", nearExtras.valsIdx))
                     return false;
 
                 writer.incrementState();
 
+            case 14:
+                if (!writer.writeMessage("nearVer", nearExtras.ver))
+                    return false;
+
+                writer.incrementState();
         }
 
         return true;
@@ -454,7 +462,7 @@ public class GridNearAtomicSingleUpdateResponse extends GridNearAtomicAbstractUp
                 reader.incrementState();
 
             case 6:
-                nearExpireTime = reader.readLong("nearExpireTime");
+                remapKey = reader.readBoolean("remapKey");
 
                 if (!reader.isLastRead())
                     return false;
@@ -462,7 +470,7 @@ public class GridNearAtomicSingleUpdateResponse extends GridNearAtomicAbstractUp
                 reader.incrementState();
 
             case 7:
-                nearSkipIdx = reader.readBoolean("nearSkipIdx");
+                ret = reader.readMessage("ret");
 
                 if (!reader.isLastRead())
                     return false;
@@ -470,15 +478,21 @@ public class GridNearAtomicSingleUpdateResponse extends GridNearAtomicAbstractUp
                 reader.incrementState();
 
             case 8:
-                nearTtl = reader.readLong("nearTtl");
+                boolean hasNearExtras = reader.readBoolean("hasNearExtras");
 
                 if (!reader.isLastRead())
                     return false;
 
+                if (hasNearExtras)
+                    nearExtras = new NearExtras();
+
                 reader.incrementState();
 
+                if (!hasNearExtras)
+                    return reader.afterMessageRead(GridNearAtomicSingleUpdateResponse.class);
+
             case 9:
-                nearVal = reader.readMessage("nearVal");
+                nearExtras.expireTime = reader.readLong("nearExpireTime");
 
                 if (!reader.isLastRead())
                     return false;
@@ -486,7 +500,7 @@ public class GridNearAtomicSingleUpdateResponse extends GridNearAtomicAbstractUp
                 reader.incrementState();
 
             case 10:
-                nearValsIdx = reader.readBoolean("nearValsIdx");
+                nearExtras.skipIdx = reader.readBoolean("nearSkipIdx");
 
                 if (!reader.isLastRead())
                     return false;
@@ -494,7 +508,7 @@ public class GridNearAtomicSingleUpdateResponse extends GridNearAtomicAbstractUp
                 reader.incrementState();
 
             case 11:
-                nearVer = reader.readMessage("nearVer");
+                nearExtras.ttl = reader.readLong("nearTtl");
 
                 if (!reader.isLastRead())
                     return false;
@@ -502,7 +516,7 @@ public class GridNearAtomicSingleUpdateResponse extends GridNearAtomicAbstractUp
                 reader.incrementState();
 
             case 12:
-                remapKey = reader.readBoolean("remapKey");
+                nearExtras.val = reader.readMessage("nearVal");
 
                 if (!reader.isLastRead())
                     return false;
@@ -510,13 +524,20 @@ public class GridNearAtomicSingleUpdateResponse extends GridNearAtomicAbstractUp
                 reader.incrementState();
 
             case 13:
-                ret = reader.readMessage("ret");
+                nearExtras.valsIdx = reader.readBoolean("nearValsIdx");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
 
+            case 14:
+                nearExtras.ver = reader.readMessage("nearVer");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
         }
 
         return reader.afterMessageRead(GridNearAtomicSingleUpdateResponse.class);
@@ -529,11 +550,35 @@ public class GridNearAtomicSingleUpdateResponse extends GridNearAtomicAbstractUp
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 14;
+        return nearExtras != null ? (byte)15 : (byte)9;
     }
 
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridNearAtomicSingleUpdateResponse.class, this, "parent");
     }
+
+    /**
+     * Extra data for near nodes.
+     */
+    private static class NearExtras {
+        /** Indexes of keys for which values were generated on primary node (used if originating
node has near cache). */
+        private boolean valsIdx;
+
+        /** Indexes of keys for which update was skipped (used if originating node has near
cache). */
+        private boolean skipIdx;
+
+        /** Values generated on primary node which should be put to originating node's near
cache. */
+        @GridToStringInclude
+        private CacheObject val;
+
+        /** Version generated on primary node to be used for originating node's near cache
update. */
+        private GridCacheVersion ver;
+
+        /** Near TTLs. */
+        private long ttl = -1L;
+
+        /** Near expire times. */
+        private long expireTime = -1L;
+    }
 }


Mime
View raw message