ignite-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From voze...@apache.org
Subject [04/51] [abbrv] ignite git commit: IGNITE-2532: WIP on single message optimization.
Date Thu, 25 Feb 2016 12:31:03 GMT
IGNITE-2532: WIP on single message optimization.


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

Branch: refs/heads/ignite-2523
Commit: 52d20cdcdc886c6ceaed49239e822a1d6deaa7dd
Parents: 2a1a31d
Author: vozerov-gridgain <vozerov@gridgain.com>
Authored: Wed Feb 3 12:03:31 2016 +0300
Committer: vozerov-gridgain <vozerov@gridgain.com>
Committed: Wed Feb 3 12:03:31 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheMessage.java      |  49 ++++
 .../GridNearAtomicSingleUpdateRequest.java      | 258 +++++++------------
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   6 +-
 3 files changed, 137 insertions(+), 176 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/52d20cdc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
index 83e3aa7..cdf579d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
@@ -463,6 +463,24 @@ public abstract class GridCacheMessage implements Message {
     }
 
     /**
+     * @param obj Object to marshal.
+     * @param ctx Context.
+     * @return Marshalled collection.
+     * @throws IgniteCheckedException If failed.
+     */
+    @Nullable protected byte[] marshal(@Nullable Object obj, GridCacheContext ctx) throws
IgniteCheckedException {
+        assert ctx != null;
+
+        if (obj == null)
+            return null;
+
+        if (addDepInfo)
+            prepareObject(obj, ctx);
+
+        return CU.marshal(ctx, obj);
+    }
+
+    /**
      * @param col Collection to marshal.
      * @param ctx Context.
      * @return Marshalled collection.
@@ -539,6 +557,19 @@ public abstract class GridCacheMessage implements Message {
     }
 
     /**
+     * @param obj Object.
+     * @param ctx Context.
+     * @param ldr Class loader.
+     * @throws IgniteCheckedException If failed.
+     */
+    @SuppressWarnings("ForLoopReplaceableByForEach")
+    protected final void finishUnmarshalCacheObject(@Nullable CacheObject obj, GridCacheContext
ctx, ClassLoader ldr)
+        throws IgniteCheckedException {
+        if (obj != null)
+            obj.finishUnmarshal(ctx.cacheObjectContext(), ldr);
+    }
+
+    /**
      * @param col Collection.
      * @param ctx Context.
      * @param ldr Class loader.
@@ -584,6 +615,24 @@ public abstract class GridCacheMessage implements Message {
     }
 
     /**
+     * @param bytes Byte array to unmarshal.
+     * @param ctx Context.
+     * @param ldr Loader.
+     * @return Unmarshalled object.
+     * @throws IgniteCheckedException If failed.
+     */
+    @Nullable protected <T> T unmarshal(@Nullable byte[] bytes, GridCacheSharedContext
ctx, ClassLoader ldr)
+        throws IgniteCheckedException {
+        assert ldr != null;
+        assert ctx != null;
+
+        if (bytes == null)
+            return null;
+
+        return ctx.marshaller().unmarshal(bytes, ldr);
+    }
+
+    /**
      * @param byteCol Collection to unmarshal.
      * @param ctx Context.
      * @param ldr Loader.

http://git-wip-us.apache.org/repos/asf/ignite/blob/52d20cdc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
index 5de9884..cee662c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
-import org.apache.ignite.internal.GridDirectCollection;
 import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheEntryPredicate;
@@ -32,7 +31,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteExternalizableExpiryPolicy;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.util.GridLongList;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -46,8 +44,8 @@ import javax.cache.expiry.ExpiryPolicy;
 import javax.cache.processor.EntryProcessor;
 import java.io.Externalizable;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import java.util.UUID;
 
@@ -91,22 +89,19 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
     /** Update operation. */
     private GridCacheOperation op;
 
-    /** Keys to update. */
+    /** Key to update. */
     @GridToStringInclude
-    @GridDirectCollection(KeyCacheObject.class)
-    private List<KeyCacheObject> keys;
+    private KeyCacheObject key;
 
-    /** Values to update. */
-    @GridDirectCollection(CacheObject.class)
-    private List<CacheObject> vals;
+    /** Value to update. */
+    private CacheObject val;
 
-    /** Entry processors. */
+    /** Entry processor. */
     @GridDirectTransient
-    private List<EntryProcessor<Object, Object, Object>> entryProcessors;
+    private EntryProcessor<Object, Object, Object> entryProc;
 
-    /** Entry processors bytes. */
-    @GridDirectCollection(byte[].class)
-    private List<byte[]> entryProcessorsBytes;
+    /** Entry processor bytes. */
+    private byte[] entryProcBytes;
 
     /** Optional arguments for entry processor. */
     @GridDirectTransient
@@ -115,15 +110,14 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
     /** Entry processor arguments bytes. */
     private byte[][] invokeArgsBytes;
 
-    /** Conflict versions. */
-    @GridDirectCollection(GridCacheVersion.class)
-    private List<GridCacheVersion> conflictVers;
+    /** Conflict version. */
+    private GridCacheVersion conflictVer;
 
-    /** Conflict TTLs. */
-    private GridLongList conflictTtls;
+    /** Conflict TTL. */
+    private long conflictTtl = CU.TTL_NOT_CHANGED;
 
-    /** Conflict expire times. */
-    private GridLongList conflictExpireTimes;
+    /** Conflict expire time. */
+    private long conflictExpireTime = CU.EXPIRE_TIME_CALCULATE;
 
     /** Return value flag. */
     private boolean retval;
@@ -138,9 +132,6 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
     /** Filter. */
     private CacheEntryPredicate[] filter;
 
-    /** Flag indicating whether request contains primary keys. */
-    private boolean hasPrimary;
-
     /** Subject ID. */
     private UUID subjId;
 
@@ -160,10 +151,6 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
     @GridDirectTransient
     private GridNearAtomicUpdateResponse res;
 
-    /** Maximum possible size of inner collections. */
-    @GridDirectTransient
-    private int initSize;
-
     /**
      * Empty constructor required by {@link Externalizable}.
      */
@@ -193,7 +180,6 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
      * @param keepBinary Keep binary flag.
      * @param clientReq Client node request flag.
      * @param addDepInfo Deployment info flag.
-     * @param maxEntryCnt Maximum entries count.
      */
     public GridNearAtomicSingleUpdateRequest(
         int cacheId,
@@ -214,8 +200,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
         boolean skipStore,
         boolean keepBinary,
         boolean clientReq,
-        boolean addDepInfo,
-        int maxEntryCnt
+        boolean addDepInfo
     ) {
         assert futVer != null;
 
@@ -239,14 +224,6 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
         this.keepBinary = keepBinary;
         this.clientReq = clientReq;
         this.addDepInfo = addDepInfo;
-
-        // By default ArrayList expands to array of 10 elements on first add. We cannot guess
how many entries
-        // will be added to request because of unknown affinity distribution. However, we
DO KNOW how many keys
-        // participate in request. As such, we know upper bound of all collections in request.
If this bound is lower
-        // than 10, we use it.
-        initSize = Math.min(maxEntryCnt, 10);
-
-        keys = new ArrayList<>(initSize);
     }
 
     /** {@inheritDoc} */
@@ -372,14 +349,13 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
      * @param conflictTtl Conflict TTL (optional).
      * @param conflictExpireTime Conflict expire time (optional).
      * @param conflictVer Conflict version (optional).
-     * @param primary If given key is primary on this mapping.
      */
+    @SuppressWarnings("unchecked")
     public void addUpdateEntry(KeyCacheObject key,
         @Nullable Object val,
         long conflictTtl,
         long conflictExpireTime,
-        @Nullable GridCacheVersion conflictVer,
-        boolean primary) {
+        @Nullable GridCacheVersion conflictVer) {
         EntryProcessor<Object, Object, Object> entryProcessor = null;
 
         if (op == TRANSFORM) {
@@ -390,74 +366,37 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
 
         assert val != null || op == DELETE;
 
-        keys.add(key);
+        this.key = key;
 
-        if (entryProcessor != null) {
-            if (entryProcessors == null)
-                entryProcessors = new ArrayList<>(initSize);
-
-            entryProcessors.add(entryProcessor);
-        }
+        if (entryProcessor != null)
+            this.entryProc = entryProcessor;
         else if (val != null) {
             assert val instanceof CacheObject : val;
 
-            if (vals == null)
-                vals = new ArrayList<>(initSize);
-
-            vals.add((CacheObject)val);
+            this.val = (CacheObject)val;
         }
 
-        hasPrimary |= primary;
+        this.conflictVer = conflictVer;
 
-        // In case there is no conflict, do not create the list.
-        if (conflictVer != null) {
-            if (conflictVers == null) {
-                conflictVers = new ArrayList<>(initSize);
-
-                for (int i = 0; i < keys.size() - 1; i++)
-                    conflictVers.add(null);
-            }
-
-            conflictVers.add(conflictVer);
-        }
-        else if (conflictVers != null)
-            conflictVers.add(null);
-
-        if (conflictTtl >= 0) {
-            if (conflictTtls == null) {
-                conflictTtls = new GridLongList(keys.size());
-
-                for (int i = 0; i < keys.size() - 1; i++)
-                    conflictTtls.add(CU.TTL_NOT_CHANGED);
-            }
-
-            conflictTtls.add(conflictTtl);
-        }
-
-        if (conflictExpireTime >= 0) {
-            if (conflictExpireTimes == null) {
-                conflictExpireTimes = new GridLongList(keys.size());
-
-                for (int i = 0; i < keys.size() - 1; i++)
-                    conflictExpireTimes.add(CU.EXPIRE_TIME_CALCULATE);
-            }
+        if (conflictTtl >= 0)
+            this.conflictTtl = conflictTtl;
 
-            conflictExpireTimes.add(conflictExpireTime);
-        }
+        if (conflictExpireTime >= 0)
+            this.conflictExpireTime = conflictExpireTime;
     }
 
     /**
      * @return Keys for this update request.
      */
     public List<KeyCacheObject> keys() {
-        return keys;
+        return Collections.singletonList(key);
     }
 
     /**
      * @return Values for this update request.
      */
     public List<?> values() {
-        return op == TRANSFORM ? entryProcessors : vals;
+        return Collections.singletonList(op == TRANSFORM ? entryProc : val);
     }
 
     /**
@@ -480,9 +419,10 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
      */
     @SuppressWarnings("unchecked")
     public CacheObject value(int idx) {
+        assert idx == 0;
         assert op == UPDATE : op;
 
-        return vals.get(idx);
+        return val;
     }
 
     /**
@@ -491,9 +431,10 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
      */
     @SuppressWarnings("unchecked")
     public EntryProcessor<Object, Object, Object> entryProcessor(int idx) {
+        assert idx == 0;
         assert op == TRANSFORM : op;
 
-        return entryProcessors.get(idx);
+        return entryProc;
     }
 
     /**
@@ -501,17 +442,16 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
      * @return Write value - either value, or transform closure.
      */
     public CacheObject writeValue(int idx) {
-        if (vals != null)
-            return vals.get(idx);
+        assert idx == 0;
 
-        return null;
+        return val;
     }
 
     /**
      * @return Conflict versions.
      */
     @Nullable public List<GridCacheVersion> conflictVersions() {
-        return conflictVers;
+        return conflictVer == null ? null : Collections.singletonList(conflictVer);
     }
 
     /**
@@ -519,13 +459,9 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
      * @return Conflict version.
      */
     @Nullable public GridCacheVersion conflictVersion(int idx) {
-        if (conflictVers != null) {
-            assert idx >= 0 && idx < conflictVers.size();
-
-            return conflictVers.get(idx);
-        }
+        assert idx == 0;
 
-        return null;
+        return conflictVer;
     }
 
     /**
@@ -533,13 +469,9 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
      * @return Conflict TTL.
      */
     public long conflictTtl(int idx) {
-        if (conflictTtls != null) {
-            assert idx >= 0 && idx < conflictTtls.size();
+        assert idx == 0;
 
-            return conflictTtls.get(idx);
-        }
-
-        return CU.TTL_NOT_CHANGED;
+        return conflictTtl;
     }
 
     /**
@@ -547,20 +479,16 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
      * @return Conflict expire time.
      */
     public long conflictExpireTime(int idx) {
-        if (conflictExpireTimes != null) {
-            assert idx >= 0 && idx < conflictExpireTimes.size();
+        assert idx == 0;
 
-            return conflictExpireTimes.get(idx);
-        }
-
-        return CU.EXPIRE_TIME_CALCULATE;
+        return conflictExpireTime;
     }
 
     /**
      * @return Flag indicating whether this request contains primary keys.
      */
     public boolean hasPrimary() {
-        return hasPrimary;
+        return true;
     }
 
     /**
@@ -591,7 +519,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
 
         GridCacheContext cctx = ctx.cacheContext(cacheId);
 
-        prepareMarshalCacheObjects(keys, cctx);
+        prepareMarshalCacheObject(key, cctx);
 
         if (filter != null) {
             boolean hasFilter = false;
@@ -616,14 +544,14 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
             if (!addDepInfo && ctx.deploymentEnabled())
                 addDepInfo = true;
 
-            if (entryProcessorsBytes == null)
-                entryProcessorsBytes = marshalCollection(entryProcessors, cctx);
+            if (entryProcBytes == null)
+                entryProcBytes = marshal(entryProc, cctx);
 
             if (invokeArgsBytes == null)
                 invokeArgsBytes = marshalInvokeArguments(invokeArgs, cctx);
         }
         else
-            prepareMarshalCacheObjects(vals, cctx);
+            prepareMarshalCacheObject(val, cctx);
     }
 
     /** {@inheritDoc} */
@@ -632,17 +560,17 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
 
         GridCacheContext cctx = ctx.cacheContext(cacheId);
 
-        finishUnmarshalCacheObjects(keys, cctx, ldr);
+        finishUnmarshalCacheObject(key, cctx, ldr);
 
         if (op == TRANSFORM) {
-            if (entryProcessors == null)
-                entryProcessors = unmarshalCollection(entryProcessorsBytes, ctx, ldr);
+            if (entryProc == null)
+                entryProc = unmarshal(entryProcBytes, ctx, ldr);
 
             if (invokeArgs == null)
                 invokeArgs = unmarshalInvokeArguments(invokeArgsBytes, ctx, ldr);
         }
         else
-            finishUnmarshalCacheObjects(vals, cctx, ldr);
+            finishUnmarshalCacheObject(val, cctx, ldr);
 
         if (filter != null) {
             for (CacheEntryPredicate p : filter) {
@@ -682,25 +610,25 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
                 writer.incrementState();
 
             case 4:
-                if (!writer.writeMessage("conflictExpireTimes", conflictExpireTimes))
+                if (!writer.writeLong("conflictExpireTime", conflictExpireTime))
                     return false;
 
                 writer.incrementState();
 
             case 5:
-                if (!writer.writeMessage("conflictTtls", conflictTtls))
+                if (!writer.writeLong("conflictTtl", conflictTtl))
                     return false;
 
                 writer.incrementState();
 
             case 6:
-                if (!writer.writeCollection("conflictVers", conflictVers, MessageCollectionItemType.MSG))
+                if (!writer.writeMessage("conflictVer", conflictVer))
                     return false;
 
                 writer.incrementState();
 
             case 7:
-                if (!writer.writeCollection("entryProcessorsBytes", entryProcessorsBytes,
MessageCollectionItemType.BYTE_ARR))
+                if (!writer.writeByteArray("entryProcBytes", entryProcBytes))
                     return false;
 
                 writer.incrementState();
@@ -730,85 +658,79 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
                 writer.incrementState();
 
             case 12:
-                if (!writer.writeBoolean("hasPrimary", hasPrimary))
+                if (!writer.writeObjectArray("invokeArgsBytes", invokeArgsBytes, MessageCollectionItemType.BYTE_ARR))
                     return false;
 
                 writer.incrementState();
 
             case 13:
-                if (!writer.writeObjectArray("invokeArgsBytes", invokeArgsBytes, MessageCollectionItemType.BYTE_ARR))
+                if (!writer.writeBoolean("keepBinary", keepBinary))
                     return false;
 
                 writer.incrementState();
 
             case 14:
-                if (!writer.writeBoolean("keepBinary", keepBinary))
+                if (!writer.writeMessage("key", key))
                     return false;
 
                 writer.incrementState();
 
             case 15:
-                if (!writer.writeCollection("keys", keys, MessageCollectionItemType.MSG))
-                    return false;
-
-                writer.incrementState();
-
-            case 16:
                 if (!writer.writeByte("op", op != null ? (byte)op.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
 
-            case 17:
+            case 16:
                 if (!writer.writeBoolean("retval", retval))
                     return false;
 
                 writer.incrementState();
 
-            case 18:
+            case 17:
                 if (!writer.writeBoolean("skipStore", skipStore))
                     return false;
 
                 writer.incrementState();
 
-            case 19:
+            case 18:
                 if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
-            case 20:
+            case 19:
                 if (!writer.writeByte("syncMode", syncMode != null ? (byte)syncMode.ordinal()
: -1))
                     return false;
 
                 writer.incrementState();
 
-            case 21:
+            case 20:
                 if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
-            case 22:
+            case 21:
                 if (!writer.writeBoolean("topLocked", topLocked))
                     return false;
 
                 writer.incrementState();
 
-            case 23:
+            case 22:
                 if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
-            case 24:
+            case 23:
                 if (!writer.writeMessage("updateVer", updateVer))
                     return false;
 
                 writer.incrementState();
 
-            case 25:
-                if (!writer.writeCollection("vals", vals, MessageCollectionItemType.MSG))
+            case 24:
+                if (!writer.writeMessage("val", val))
                     return false;
 
                 writer.incrementState();
@@ -838,7 +760,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
                 reader.incrementState();
 
             case 4:
-                conflictExpireTimes = reader.readMessage("conflictExpireTimes");
+                conflictExpireTime = reader.readLong("conflictExpireTime");
 
                 if (!reader.isLastRead())
                     return false;
@@ -846,7 +768,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
                 reader.incrementState();
 
             case 5:
-                conflictTtls = reader.readMessage("conflictTtls");
+                conflictTtl = reader.readLong("conflictTtl");
 
                 if (!reader.isLastRead())
                     return false;
@@ -854,7 +776,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
                 reader.incrementState();
 
             case 6:
-                conflictVers = reader.readCollection("conflictVers", MessageCollectionItemType.MSG);
+                conflictVer = reader.readMessage("conflictVer");
 
                 if (!reader.isLastRead())
                     return false;
@@ -862,7 +784,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
                 reader.incrementState();
 
             case 7:
-                entryProcessorsBytes = reader.readCollection("entryProcessorsBytes", MessageCollectionItemType.BYTE_ARR);
+                entryProcBytes = reader.readByteArray("entryProcBytes");
 
                 if (!reader.isLastRead())
                     return false;
@@ -902,7 +824,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
                 reader.incrementState();
 
             case 12:
-                hasPrimary = reader.readBoolean("hasPrimary");
+                invokeArgsBytes = reader.readObjectArray("invokeArgsBytes", MessageCollectionItemType.BYTE_ARR,
byte[].class);
 
                 if (!reader.isLastRead())
                     return false;
@@ -910,7 +832,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
                 reader.incrementState();
 
             case 13:
-                invokeArgsBytes = reader.readObjectArray("invokeArgsBytes", MessageCollectionItemType.BYTE_ARR,
byte[].class);
+                keepBinary = reader.readBoolean("keepBinary");
 
                 if (!reader.isLastRead())
                     return false;
@@ -918,7 +840,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
                 reader.incrementState();
 
             case 14:
-                keepBinary = reader.readBoolean("keepBinary");
+                key = reader.readMessage("key");
 
                 if (!reader.isLastRead())
                     return false;
@@ -926,14 +848,6 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
                 reader.incrementState();
 
             case 15:
-                keys = reader.readCollection("keys", MessageCollectionItemType.MSG);
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 16:
                 byte opOrd;
 
                 opOrd = reader.readByte("op");
@@ -945,7 +859,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
 
                 reader.incrementState();
 
-            case 17:
+            case 16:
                 retval = reader.readBoolean("retval");
 
                 if (!reader.isLastRead())
@@ -953,7 +867,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
 
                 reader.incrementState();
 
-            case 18:
+            case 17:
                 skipStore = reader.readBoolean("skipStore");
 
                 if (!reader.isLastRead())
@@ -961,7 +875,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
 
                 reader.incrementState();
 
-            case 19:
+            case 18:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -969,7 +883,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
 
                 reader.incrementState();
 
-            case 20:
+            case 19:
                 byte syncModeOrd;
 
                 syncModeOrd = reader.readByte("syncMode");
@@ -981,7 +895,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
 
                 reader.incrementState();
 
-            case 21:
+            case 20:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -989,7 +903,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
 
                 reader.incrementState();
 
-            case 22:
+            case 21:
                 topLocked = reader.readBoolean("topLocked");
 
                 if (!reader.isLastRead())
@@ -997,7 +911,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
 
                 reader.incrementState();
 
-            case 23:
+            case 22:
                 topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
@@ -1005,7 +919,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
 
                 reader.incrementState();
 
-            case 24:
+            case 23:
                 updateVer = reader.readMessage("updateVer");
 
                 if (!reader.isLastRead())
@@ -1013,8 +927,8 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
 
                 reader.incrementState();
 
-            case 25:
-                vals = reader.readCollection("vals", MessageCollectionItemType.MSG);
+            case 24:
+                val = reader.readMessage("val");
 
                 if (!reader.isLastRead())
                     return false;
@@ -1033,7 +947,7 @@ public class GridNearAtomicSingleUpdateRequest extends GridCacheMessage
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 26;
+        return 25;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/52d20cdc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
index 493c765..38e93ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
@@ -1028,15 +1028,13 @@ public class GridNearAtomicUpdateFuture extends GridNearAbstractAtomicUpdateFutu
                 skipStore,
                 keepBinary,
                 cctx.kernalContext().clientNode(),
-                cctx.deploymentEnabled(),
-                1);
+                cctx.deploymentEnabled());
 
             req.addUpdateEntry(cacheKey,
                 val,
                 conflictTtl,
                 conflictExpireTime,
-                conflictVer,
-                true);
+                conflictVer);
 
             return req;
         }


Mime
View raw message