You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2015/11/05 17:28:57 UTC

[37/50] [abbrv] ignite git commit: Merged IGNITE-950-new into IGNITE-1282

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 1f4852c..e244aa5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -1710,7 +1710,7 @@ public class GridCacheContext<K, V> implements Externalizable {
     public boolean keepPortable() {
         CacheOperationContext opCtx = operationContextPerCall();
 
-        return opCtx != null && opCtx.isKeepPortable();
+        return opCtx != null && opCtx.isKeepBinary();
     }
 
     /**
@@ -1752,7 +1752,7 @@ public class GridCacheContext<K, V> implements Externalizable {
     }
 
     /**
-     * Unwraps object for portables.
+     * Unwraps object for binary.
      *
      * @param o Object to unwrap.
      * @param keepPortable Keep portable flag.
@@ -1855,15 +1855,9 @@ public class GridCacheContext<K, V> implements Externalizable {
         assert val != null || skipVals;
 
         if (!keepCacheObjects) {
-            Object key0 = key.value(cacheObjCtx, false);
-            Object val0 = skipVals ? true : val.value(cacheObjCtx, cpy);
+            Object key0 = unwrapPortableIfNeeded(key, !deserializePortable);
 
-            if (deserializePortable) {
-                key0 = unwrapPortableIfNeeded(key0, false);
-
-                if (!skipVals)
-                    val0 = unwrapPortableIfNeeded(val0, false);
-            }
+            Object val0 = skipVals ? true : unwrapPortableIfNeeded(val, !deserializePortable);
 
             assert key0 != null : key;
             assert val0 != null : val;

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
index 50b01c8..81914bb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
@@ -303,7 +303,8 @@ public interface GridCacheEntryEx {
         UUID subjId,
         Object transformClo,
         String taskName,
-        @Nullable IgniteCacheExpiryPolicy expiryPlc)
+        @Nullable IgniteCacheExpiryPolicy expiryPlc,
+        boolean keepBinary)
         throws IgniteCheckedException, GridCacheEntryRemovedException;
 
     /**
@@ -329,7 +330,8 @@ public interface GridCacheEntryEx {
         UUID subjId,
         Object transformClo,
         String taskName,
-        @Nullable IgniteCacheExpiryPolicy expiryPlc)
+        @Nullable IgniteCacheExpiryPolicy expiryPlc,
+        boolean keepBinary)
         throws IgniteCheckedException, GridCacheEntryRemovedException;
 
     /**
@@ -375,6 +377,7 @@ public interface GridCacheEntryEx {
         long ttl,
         boolean evt,
         boolean metrics,
+        boolean keepPortable,
         AffinityTopologyVersion topVer,
         CacheEntryPredicate[] filter,
         GridDrType drType,
@@ -413,6 +416,7 @@ public interface GridCacheEntryEx {
         boolean retval,
         boolean evt,
         boolean metrics,
+        boolean keepPortable,
         AffinityTopologyVersion topVer,
         CacheEntryPredicate[] filter,
         GridDrType drType,
@@ -466,6 +470,7 @@ public interface GridCacheEntryEx {
         boolean writeThrough,
         boolean readThrough,
         boolean retval,
+        boolean keepPortable,
         @Nullable IgniteCacheExpiryPolicy expiryPlc,
         boolean evt,
         boolean metrics,
@@ -512,6 +517,7 @@ public interface GridCacheEntryEx {
         boolean writeThrough,
         boolean readThrough,
         boolean retval,
+        boolean keepPortable,
         @Nullable ExpiryPolicy expiryPlc,
         boolean evt,
         boolean metrics,
@@ -549,8 +555,9 @@ public interface GridCacheEntryEx {
     public boolean tmLock(IgniteInternalTx tx,
         long timeout,
         @Nullable GridCacheVersion serOrder,
-        @Nullable GridCacheVersion serReadVer)
-        throws GridCacheEntryRemovedException, GridDistributedLockCancelledException;
+        @Nullable GridCacheVersion serReadVer,
+        boolean keepBinary
+    ) throws GridCacheEntryRemovedException, GridDistributedLockCancelledException;
 
     /**
      * Unlocks acquired lock.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
index 751c316..afca43b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
@@ -26,7 +26,6 @@ import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.X;
-import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
@@ -84,7 +83,8 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
         boolean hasOldVal,
         UUID subjId,
         String cloClsName,
-        String taskName)
+        String taskName,
+        boolean keepBinary)
     {
         addEvent(part,
             key,
@@ -98,7 +98,8 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
             hasOldVal,
             subjId,
             cloClsName,
-            taskName);
+            taskName,
+            keepBinary);
     }
 
     /**
@@ -118,7 +119,8 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
             false,
             null,
             null,
-            null);
+            null,
+            false);
     }
 
     /**
@@ -148,7 +150,8 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
         boolean hasOldVal,
         UUID subjId,
         String cloClsName,
-        String taskName)
+        String taskName,
+        boolean keepPortable)
     {
         addEvent(part,
             key,
@@ -161,7 +164,8 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
             hasOldVal,
             subjId,
             cloClsName,
-            taskName);
+            taskName,
+            keepPortable);
     }
 
     /**
@@ -189,7 +193,8 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
         boolean hasOldVal,
         UUID subjId,
         String cloClsName,
-        String taskName)
+        String taskName,
+        boolean keepBinary)
     {
         IgniteInternalTx tx = owner == null ? null : cctx.tm().tx(owner.version());
 
@@ -205,7 +210,8 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
             hasOldVal,
             subjId,
             cloClsName,
-            taskName);
+            taskName,
+            keepBinary);
     }
 
     /**
@@ -236,7 +242,8 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
         boolean hasOldVal,
         UUID subjId,
         @Nullable String cloClsName,
-        @Nullable String taskName
+        @Nullable String taskName,
+        boolean keepPortable
     ) {
         assert key != null || type == EVT_CACHE_STARTED || type == EVT_CACHE_STOPPED;
 
@@ -262,12 +269,12 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
                 type,
                 part,
                 cctx.isNear(),
-                key == null ? null : key.value(cctx.cacheObjectContext(), false),
+                cctx.cacheObjectContext().unwrapPortableIfNeeded(key, keepPortable, false),
                 xid,
                 lockId,
-                CU.value(newVal, cctx, false),
+                cctx.cacheObjectContext().unwrapPortableIfNeeded(newVal, keepPortable, false),
                 hasNewVal,
-                CU.value(oldVal, cctx, false),
+                cctx.cacheObjectContext().unwrapPortableIfNeeded(oldVal, keepPortable, false),
                 hasOldVal,
                 subjId,
                 cloClsName,

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
index 9a89fee..845e204 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
@@ -723,7 +723,7 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter {
 
             if (recordable)
                 cctx.events().addEvent(entry.partition(), entry.key(), cctx.nodeId(), (IgniteUuid)null, null,
-                    EVT_CACHE_ENTRY_EVICTED, null, false, oldVal, hasVal, null, null, null);
+                    EVT_CACHE_ENTRY_EVICTED, null, false, oldVal, hasVal, null, null, null, false);
 
             if (log.isDebugEnabled())
                 log.debug("Entry was evicted [entry=" + entry + ", localNode=" + cctx.nodeId() + ']');
@@ -1043,7 +1043,8 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter {
 
                     if (recordable)
                         cctx.events().addEvent(entry.partition(), entry.key(), cctx.nodeId(), (IgniteUuid)null, null,
-                            EVT_CACHE_ENTRY_EVICTED, null, false, entry.rawGet(), entry.hasValue(), null, null, null);
+                            EVT_CACHE_ENTRY_EVICTED, null, false, entry.rawGet(), entry.hasValue(), null, null, null,
+                            false);
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 2111594..a5762ad 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -234,7 +234,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 if (cctx.kernalContext().config().isPeerClassLoadingEnabled()) {
                     Object val0 = null;
 
-                    if (val != null && val.type() != CacheObject.TYPE_BYTE_ARR) {
+                    if (val != null && val.cacheObjectType() != CacheObject.TYPE_BYTE_ARR) {
                         val0 = cctx.cacheObjects().unmarshal(cctx.cacheObjectContext(),
                             val.valueBytes(cctx.cacheObjectContext()), cctx.deploy().globalLoader());
 
@@ -253,7 +253,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 assert mem != null;
 
                 if (val != null) {
-                    byte type = val.type();
+                    byte type = val.cacheObjectType();
 
                     offHeapPointer(mem.putOffHeap(offHeapPointer(), val.valueBytes(cctx.cacheObjectContext()), type));
                 }
@@ -625,7 +625,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             try {
                 byte[] bytes = val.valueBytes(cctx.cacheObjectContext());
 
-                return new IgniteBiTuple<>(bytes, val.type());
+                return new IgniteBiTuple<>(bytes, val.cacheObjectType());
             }
             catch (IgniteCheckedException e) {
                 throw new IgniteException(e);
@@ -674,7 +674,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         UUID subjId,
         Object transformClo,
         String taskName,
-        @Nullable IgniteCacheExpiryPolicy expirePlc)
+        @Nullable IgniteCacheExpiryPolicy expirePlc,
+        boolean keepBinary)
         throws IgniteCheckedException, GridCacheEntryRemovedException {
         return (CacheObject)innerGet0(tx,
             readSwap,
@@ -687,7 +688,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             transformClo,
             taskName,
             expirePlc,
-            false);
+            false,
+            keepBinary);
     }
 
     /** {@inheritDoc} */
@@ -700,7 +702,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         UUID subjId,
         Object transformClo,
         String taskName,
-        @Nullable IgniteCacheExpiryPolicy expiryPlc)
+        @Nullable IgniteCacheExpiryPolicy expiryPlc,
+        boolean keepBinary)
         throws IgniteCheckedException, GridCacheEntryRemovedException {
         return (T2<CacheObject, GridCacheVersion>)innerGet0(tx,
             readSwap,
@@ -713,7 +716,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             transformClo,
             taskName,
             expiryPlc,
-            true);
+            true,
+            keepBinary);
     }
 
     /** {@inheritDoc} */
@@ -729,8 +733,9 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         Object transformClo,
         String taskName,
         @Nullable IgniteCacheExpiryPolicy expiryPlc,
-        boolean retVer)
-        throws IgniteCheckedException, GridCacheEntryRemovedException {
+        boolean retVer,
+        boolean keepBinary
+    ) throws IgniteCheckedException, GridCacheEntryRemovedException {
         assert !(retVer && readThrough);
 
         // Disable read-through if there is no store.
@@ -854,7 +859,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                         expiredVal != null || hasOldBytes,
                         subjId,
                         null,
-                        taskName);
+                        taskName,
+                        keepBinary);
                 }
 
                 cctx.continuousQueries().onEntryExpired(this, key, expiredVal);
@@ -864,9 +870,20 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             }
 
             if (evt && !expired && cctx.events().isRecordable(EVT_CACHE_OBJECT_READ)) {
-                cctx.events().addEvent(partition(), key, tx, owner, EVT_CACHE_OBJECT_READ, ret, ret != null, old,
-                    hasOldBytes || old != null, subjId,
-                    transformClo != null ? transformClo.getClass().getName() : null, taskName);
+                cctx.events().addEvent(
+                    partition(),
+                    key,
+                    tx,
+                    owner,
+                    EVT_CACHE_OBJECT_READ,
+                    ret,
+                    ret != null,
+                    old,
+                    hasOldBytes || old != null,
+                    subjId,
+                    transformClo != null ? transformClo.getClass().getName() : null,
+                    taskName,
+                    keepBinary);
 
                 // No more notifications.
                 evt = false;
@@ -941,9 +958,20 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 }
 
                 if (evt && cctx.events().isRecordable(EVT_CACHE_OBJECT_READ))
-                    cctx.events().addEvent(partition(), key, tx, owner, EVT_CACHE_OBJECT_READ, ret, ret != null,
-                        old, hasOldBytes, subjId, transformClo != null ? transformClo.getClass().getName() : null,
-                        taskName);
+                    cctx.events().addEvent(
+                        partition(),
+                        key,
+                        tx,
+                        owner,
+                        EVT_CACHE_OBJECT_READ,
+                        ret,
+                        ret != null,
+                        old,
+                        hasOldBytes,
+                        subjId,
+                        transformClo != null ? transformClo.getClass().getName() : null,
+                        taskName,
+                        keepBinary);
             }
         }
 
@@ -1052,6 +1080,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         long ttl,
         boolean evt,
         boolean metrics,
+        boolean keepPortable,
         AffinityTopologyVersion topVer,
         CacheEntryPredicate[] filter,
         GridDrType drType,
@@ -1106,9 +1135,9 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             if (intercept) {
                 val0 = CU.value(val, cctx, false);
 
-                CacheLazyEntry e = new CacheLazyEntry(cctx, key, old);
+                CacheLazyEntry e = new CacheLazyEntry(cctx, key, old, keepPortable);
 
-                Object interceptorVal = cctx.config().getInterceptor().onBeforePut(new CacheLazyEntry(cctx, key, old),
+                Object interceptorVal = cctx.config().getInterceptor().onBeforePut(new CacheLazyEntry(cctx, key, old, keepPortable),
                     val0);
 
                 key0 = e.key();
@@ -1175,13 +1204,14 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                     val != null,
                     evtOld,
                     evtOld != null || hasValueUnlocked(),
-                    subjId, null, taskName);
+                    subjId, null, taskName,
+                    keepPortable);
             }
 
             if (cctx.isLocal() || cctx.isReplicated() || (tx != null && tx.local() && !isNear()))
                 cctx.continuousQueries().onEntryUpdated(this, key, val, old, false);
 
-            cctx.dataStructures().onEntryUpdated(key, false);
+            cctx.dataStructures().onEntryUpdated(key, false, keepPortable);
         }
 
         if (log.isDebugEnabled())
@@ -1193,7 +1223,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             cctx.store().put(tx, keyValue(false), CU.value(val, cctx, false), newVer);
 
         if (intercept)
-            cctx.config().getInterceptor().onAfterPut(new CacheLazyEntry(cctx, key, key0, val, val0));
+            cctx.config().getInterceptor().onAfterPut(new CacheLazyEntry(cctx, key, key0, val, val0, keepPortable));
 
         return valid ? new GridCacheUpdateTxResult(true, retval ? old : null) :
             new GridCacheUpdateTxResult(false, null);
@@ -1216,6 +1246,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         boolean retval,
         boolean evt,
         boolean metrics,
+        boolean keepPortable,
         AffinityTopologyVersion topVer,
         CacheEntryPredicate[] filter,
         GridDrType drType,
@@ -1270,7 +1301,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             old = (retval || intercept) ? rawGetOrUnmarshalUnlocked(!retval) : val;
 
             if (intercept) {
-                entry0 = new CacheLazyEntry(cctx, key, old);
+                entry0 = new CacheLazyEntry(cctx, key, old, keepPortable);
 
                 interceptRes = cctx.config().getInterceptor().onBeforeRemove(entry0);
 
@@ -1342,13 +1373,14 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                     evtOld != null || hasValueUnlocked(),
                     subjId,
                     null,
-                    taskName);
+                    taskName,
+                    keepPortable);
             }
 
             if (cctx.isLocal() || cctx.isReplicated() || (tx != null && tx.local() && !isNear()))
                 cctx.continuousQueries().onEntryUpdated(this, key, null, old, false);
 
-            cctx.dataStructures().onEntryUpdated(key, true);
+            cctx.dataStructures().onEntryUpdated(key, true, keepPortable);
         }
 
         // Persist outside of synchronization. The correctness of the
@@ -1412,6 +1444,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         boolean writeThrough,
         boolean readThrough,
         boolean retval,
+        boolean keepBinary,
         @Nullable ExpiryPolicy expiryPlc,
         boolean evt,
         boolean metrics,
@@ -1513,7 +1546,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 assert entryProcessor != null;
 
-                CacheInvokeEntry<Object, Object> entry = new CacheInvokeEntry<>(cctx, key, old, version());
+                CacheInvokeEntry<Object, Object> entry = new CacheInvokeEntry<>(cctx, key, old, version(), keepBinary);
 
                 try {
                     Object computed = entryProcessor.process(entry, invokeArgs);
@@ -1554,7 +1587,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 if (op == GridCacheOperation.UPDATE) {
                     updated0 = value(updated0, updated, false);
 
-                    e = new CacheLazyEntry(cctx, key, key0, old, old0);
+                    e = new CacheLazyEntry(cctx, key, key0, old, old0, keepBinary);
 
                     Object interceptorVal = cctx.config().getInterceptor().onBeforePut(e, updated0);
 
@@ -1567,7 +1600,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                     }
                 }
                 else {
-                    e = new CacheLazyEntry(cctx, key, key0, old, old0);
+                    e = new CacheLazyEntry(cctx, key, key0, old, old0, keepBinary);
 
                     interceptorRes = cctx.config().getInterceptor().onBeforeRemove(e);
 
@@ -1629,7 +1662,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                         cctx.events().addEvent(partition(), key, cctx.localNodeId(), null,
                             (GridCacheVersion)null, EVT_CACHE_OBJECT_READ, evtOld, evtOld != null || hadVal, evtOld,
-                            evtOld != null || hadVal, subjId, transformCloClsName, taskName);
+                            evtOld != null || hadVal, subjId, transformCloClsName, taskName, keepBinary);
                     }
 
                     if (cctx.events().isRecordable(EVT_CACHE_OBJECT_PUT)) {
@@ -1638,7 +1671,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                         cctx.events().addEvent(partition(), key, cctx.localNodeId(), null,
                             (GridCacheVersion)null, EVT_CACHE_OBJECT_PUT, updated, updated != null, evtOld,
-                            evtOld != null || hadVal, subjId, null, taskName);
+                            evtOld != null || hadVal, subjId, null, taskName, keepBinary);
                     }
                 }
             }
@@ -1670,7 +1703,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                     if (transformCloClsName != null && cctx.events().isRecordable(EVT_CACHE_OBJECT_READ))
                         cctx.events().addEvent(partition(), key, cctx.localNodeId(), null,
                             (GridCacheVersion)null, EVT_CACHE_OBJECT_READ, evtOld, evtOld != null || hadVal, evtOld,
-                            evtOld != null || hadVal, subjId, transformCloClsName, taskName);
+                            evtOld != null || hadVal, subjId, transformCloClsName, taskName, keepBinary);
 
                     if (cctx.events().isRecordable(EVT_CACHE_OBJECT_REMOVED)) {
                         if (evtOld == null)
@@ -1678,7 +1711,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                         cctx.events().addEvent(partition(), key, cctx.localNodeId(), null, (GridCacheVersion)null,
                             EVT_CACHE_OBJECT_REMOVED, null, false, evtOld, evtOld != null || hadVal, subjId, null,
-                            taskName);
+                            taskName, keepBinary);
                     }
                 }
 
@@ -1690,18 +1723,20 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
             cctx.continuousQueries().onEntryUpdated(this, key, val, old, false);
 
-            cctx.dataStructures().onEntryUpdated(key, op == GridCacheOperation.DELETE);
+            cctx.dataStructures().onEntryUpdated(key, op == GridCacheOperation.DELETE, keepBinary);
 
             if (intercept) {
                 if (op == GridCacheOperation.UPDATE)
-                    cctx.config().getInterceptor().onAfterPut(new CacheLazyEntry(cctx, key, key0, updated, updated0));
+                    cctx.config().getInterceptor().onAfterPut(new CacheLazyEntry(cctx, key, key0, updated, updated0, keepBinary));
                 else
-                    cctx.config().getInterceptor().onAfterRemove(new CacheLazyEntry(cctx, key, key0, old, old0));
+                    cctx.config().getInterceptor().onAfterRemove(new CacheLazyEntry(cctx, key, key0, old, old0, keepBinary));
             }
         }
 
         return new GridTuple3<>(res,
-            cctx.unwrapTemporary(interceptorRes != null ? interceptorRes.get2() : CU.value(old, cctx, false)),
+            cctx.unwrapTemporary(interceptorRes != null ?
+                interceptorRes.get2() :
+                cctx.cacheObjectContext().unwrapPortableIfNeeded(old, keepBinary, false)),
             invokeRes);
     }
 
@@ -1717,6 +1752,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         boolean writeThrough,
         boolean readThrough,
         boolean retval,
+        boolean keepPortable,
         @Nullable IgniteCacheExpiryPolicy expiryPlc,
         boolean evt,
         boolean metrics,
@@ -1784,7 +1820,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                         oldVal = rawGetOrUnmarshalUnlocked(true);
 
-                        CacheInvokeEntry<Object, Object> entry = new CacheInvokeEntry(cctx, key, oldVal, version());
+                        CacheInvokeEntry<Object, Object> entry = new CacheInvokeEntry(cctx, key, oldVal, version(), keepPortable);
 
                         try {
                             Object computed = entryProcessor.process(entry, invokeArgs);
@@ -2009,7 +2045,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 EntryProcessor<Object, Object, ?> entryProcessor = (EntryProcessor<Object, Object, ?>)writeObj;
 
-                CacheInvokeEntry<Object, Object> entry = new CacheInvokeEntry(cctx, key, oldVal, version());
+                CacheInvokeEntry<Object, Object> entry = new CacheInvokeEntry(cctx, key, oldVal, version(), keepPortable);
 
                 try {
                     Object computed = entryProcessor.process(entry, invokeArgs);
@@ -2133,7 +2169,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                     updated0 = value(updated0, updated, false);
 
                     Object interceptorVal = cctx.config().getInterceptor()
-                        .onBeforePut(new CacheLazyEntry(cctx, key, key0, oldVal, old0), updated0);
+                        .onBeforePut(new CacheLazyEntry(cctx, key, key0, oldVal, old0, keepPortable), updated0);
 
                     if (interceptorVal == null)
                         return new GridCacheUpdateAtomicResult(false,
@@ -2155,7 +2191,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 // Try write-through.
                 if (writeThrough)
                     // Must persist inside synchronization in non-tx mode.
-                    cctx.store().put(null, keyValue(false), CU.value(updated, cctx, false), newVer);
+                    cctx.store().put(null, key, updated, newVer);
 
                 if (!hadVal) {
                     boolean new0 = isNew();
@@ -2193,7 +2229,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                         cctx.events().addEvent(partition(), key, evtNodeId, null,
                             newVer, EVT_CACHE_OBJECT_READ, evtOld, evtOld != null || hadVal, evtOld,
-                            evtOld != null || hadVal, subjId, transformClo.getClass().getName(), taskName);
+                            evtOld != null || hadVal, subjId, transformClo.getClass().getName(), taskName,
+                            keepPortable);
                     }
 
                     if (newVer != null && cctx.events().isRecordable(EVT_CACHE_OBJECT_PUT)) {
@@ -2202,14 +2239,14 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                         cctx.events().addEvent(partition(), key, evtNodeId, null,
                             newVer, EVT_CACHE_OBJECT_PUT, updated, updated != null, evtOld,
-                            evtOld != null || hadVal, subjId, null, taskName);
+                            evtOld != null || hadVal, subjId, null, taskName, keepPortable);
                     }
                 }
             }
             else {
                 if (intercept) {
                     interceptRes = cctx.config().getInterceptor().onBeforeRemove(new CacheLazyEntry(cctx, key, key0,
-                        oldVal, old0));
+                        oldVal, old0, keepPortable));
 
                     if (cctx.cancelRemove(interceptRes))
                         return new GridCacheUpdateAtomicResult(false,
@@ -2225,7 +2262,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 if (writeThrough)
                     // Must persist inside synchronization in non-tx mode.
-                    cctx.store().remove(null, keyValue(false));
+                    cctx.store().remove(null, key);
 
                 if (oldVal == null)
                     oldVal = saveValueForIndexUnlocked();
@@ -2282,7 +2319,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                         cctx.events().addEvent(partition(), key, evtNodeId, null,
                             newVer, EVT_CACHE_OBJECT_READ, evtOld, evtOld != null || hadVal, evtOld,
-                            evtOld != null || hadVal, subjId, transformClo.getClass().getName(), taskName);
+                            evtOld != null || hadVal, subjId, transformClo.getClass().getName(), taskName,
+                            keepPortable);
                     }
 
                     if (newVer != null && cctx.events().isRecordable(EVT_CACHE_OBJECT_REMOVED)) {
@@ -2291,7 +2329,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                         cctx.events().addEvent(partition(), key, evtNodeId, null, newVer,
                             EVT_CACHE_OBJECT_REMOVED, null, false, evtOld, evtOld != null || hadVal,
-                            subjId, null, taskName);
+                            subjId, null, taskName, keepPortable);
                     }
                 }
 
@@ -2304,13 +2342,13 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             if (cctx.isReplicated() || primary)
                 cctx.continuousQueries().onEntryUpdated(this, key, val, oldVal, false);
 
-            cctx.dataStructures().onEntryUpdated(key, op == GridCacheOperation.DELETE);
+            cctx.dataStructures().onEntryUpdated(key, op == GridCacheOperation.DELETE, keepPortable);
 
             if (intercept) {
                 if (op == GridCacheOperation.UPDATE)
-                    cctx.config().getInterceptor().onAfterPut(new CacheLazyEntry(cctx, key, key0, updated, updated0));
+                    cctx.config().getInterceptor().onAfterPut(new CacheLazyEntry(cctx, key, key0, updated, updated0, keepPortable));
                 else
-                    cctx.config().getInterceptor().onAfterRemove(new CacheLazyEntry(cctx, key, key0, oldVal, old0));
+                    cctx.config().getInterceptor().onAfterRemove(new CacheLazyEntry(cctx, key, key0, oldVal, old0, keepPortable));
 
                 if (interceptRes != null)
                     oldVal = cctx.toCacheObject(cctx.unwrapTemporary(interceptRes.get2()));
@@ -3153,7 +3191,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                     if (cctx.isLocal() || cctx.isReplicated() || cctx.affinity().primary(cctx.localNode(), key, topVer))
                         cctx.continuousQueries().onEntryUpdated(this, key, val, null, preload);
 
-                    cctx.dataStructures().onEntryUpdated(key, false);
+                    cctx.dataStructures().onEntryUpdated(key, false, true);
                 }
 
                 if (cctx.store().isLocal()) {
@@ -3486,7 +3524,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                             expiredVal != null || hasOldBytes,
                             null,
                             null,
-                            null);
+                            null,
+                            true);
                     }
 
                     cctx.continuousQueries().onEntryExpired(this, key, expiredVal);
@@ -3715,7 +3754,9 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
     /** {@inheritDoc} */
     @Override public <K, V> Cache.Entry<K, V> wrapLazyValue() {
-        return new LazyValueEntry<>(key);
+        CacheOperationContext opCtx = cctx.operationContextPerCall();
+
+        return new LazyValueEntry<>(key, opCtx != null && opCtx.isKeepBinary());
     }
 
     /** {@inheritDoc} */
@@ -4196,22 +4237,26 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         /** */
         private final KeyCacheObject key;
 
+        /** */
+        private boolean keepBinary;
+
         /**
          * @param key Key.
          */
-        private LazyValueEntry(KeyCacheObject key) {
+        private LazyValueEntry(KeyCacheObject key, boolean keepBinary) {
             this.key = key;
+            this.keepBinary = keepBinary;
         }
 
         /** {@inheritDoc} */
         @Override public K getKey() {
-            return key.value(cctx.cacheObjectContext(), false);
+            return (K)cctx.cacheObjectContext().unwrapPortableIfNeeded(key, keepBinary);
         }
 
         /** {@inheritDoc} */
         @SuppressWarnings("unchecked")
         @Override public V getValue() {
-            return CU.value(peekVisibleValue(), cctx, true);
+            return (V)cctx.cacheObjectContext().unwrapPortableIfNeeded(peekVisibleValue(), keepBinary);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
index cd779f2..cca6d6a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
@@ -229,12 +229,12 @@ public class GridCacheProxyImpl<K, V> implements IgniteInternalCache<K, V>, Exte
 
     /** {@inheritDoc} */
     @Override public <K1, V1> GridCacheProxyImpl<K1, V1> keepPortable() {
-        if (opCtx != null && opCtx.isKeepPortable())
+        if (opCtx != null && opCtx.isKeepBinary())
             return (GridCacheProxyImpl<K1, V1>)this;
 
         return new GridCacheProxyImpl<>((GridCacheContext<K1, V1>)ctx,
             (GridCacheAdapter<K1, V1>)delegate,
-            opCtx != null ? opCtx.keepPortable() : new CacheOperationContext(false, null, true, null, false));
+            opCtx != null ? opCtx.keepBinary() : new CacheOperationContext(false, null, true, null, false));
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturn.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturn.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturn.java
index 22b9f8d..2d179fa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturn.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturn.java
@@ -101,13 +101,13 @@ public class GridCacheReturn implements Externalizable, Message {
      * @param v Value.
      * @param success Success flag.
      */
-    public GridCacheReturn(GridCacheContext cctx, boolean loc, Object v, boolean success) {
+    public GridCacheReturn(GridCacheContext cctx, boolean loc, boolean keepBinary, Object v, boolean success) {
         this.loc = loc;
         this.success = success;
 
         if (v != null) {
             if (v instanceof CacheObject)
-                initValue(cctx, (CacheObject)v);
+                initValue(cctx, (CacheObject)v, keepBinary);
             else {
                 assert loc;
 
@@ -152,8 +152,8 @@ public class GridCacheReturn implements Externalizable, Message {
      * @param v Value.
      * @return This instance for chaining.
      */
-    public GridCacheReturn value(GridCacheContext cctx, CacheObject v) {
-        initValue(cctx, v);
+    public GridCacheReturn value(GridCacheContext cctx, CacheObject v, boolean keepBinary) {
+        initValue(cctx, v, keepBinary);
 
         return this;
     }
@@ -171,10 +171,15 @@ public class GridCacheReturn implements Externalizable, Message {
      * @param success Success flag to set.
      * @return This instance for chaining.
      */
-    public GridCacheReturn set(GridCacheContext cctx, @Nullable CacheObject cacheObj, boolean success) {
+    public GridCacheReturn set(
+        GridCacheContext cctx,
+        @Nullable CacheObject cacheObj,
+        boolean success,
+        boolean keepBinary
+    ) {
         this.success = success;
 
-        initValue(cctx, cacheObj);
+        initValue(cctx, cacheObj, keepBinary);
 
         return this;
     }
@@ -183,9 +188,9 @@ public class GridCacheReturn implements Externalizable, Message {
      * @param cctx Cache context.
      * @param cacheObj Cache object.
      */
-    private void initValue(GridCacheContext cctx, @Nullable CacheObject cacheObj) {
+    private void initValue(GridCacheContext cctx, @Nullable CacheObject cacheObj, boolean keepBinary) {
         if (loc)
-            v = CU.value(cacheObj, cctx, true);
+            v = cctx.cacheObjectContext().unwrapPortableIfNeeded(cacheObj, keepBinary, true);
         else {
             assert cacheId == 0 || cacheId == cctx.cacheId();
 
@@ -313,7 +318,7 @@ public class GridCacheReturn implements Externalizable, Message {
         if (cacheObj != null) {
             cacheObj.finishUnmarshal(ctx.cacheObjectContext(), ldr);
 
-            v = cacheObj.value(ctx.cacheObjectContext(), false);
+            v = ctx.cacheObjectContext().unwrapPortableIfNeeded(cacheObj, true, false);
         }
 
         if (invokeRes && invokeResCol != null) {
@@ -324,10 +329,10 @@ public class GridCacheReturn implements Externalizable, Message {
 
             for (CacheInvokeDirectResult res : invokeResCol) {
                 CacheInvokeResult<?> res0 = res.error() == null ?
-                    CacheInvokeResult.fromResult(CU.value(res.result(), ctx, false)) :
+                    CacheInvokeResult.fromResult(ctx.cacheObjectContext().unwrapPortableIfNeeded(res.result(), true, false)) :
                     CacheInvokeResult.fromError(res.error());
 
-                map0.put(res.key().value(ctx.cacheObjectContext(), false), res0);
+                map0.put(ctx.cacheObjectContext().unwrapPortableIfNeeded(res.key(), true, false), res0);
             }
 
             v = map0;

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
index b9b7a30..37b5e15 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
@@ -550,7 +550,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
             assert key != null;
 
             byte[] valBytes = swapMgr.read(spaceName,
-                new SwapKey(key.value(cctx.cacheObjectContext(), false), part, key.valueBytes(cctx.cacheObjectContext())),
+                new SwapKey(key, part, key.valueBytes(cctx.cacheObjectContext())),
                 cctx.deploy().globalLoader());
 
             if (cctx.config().isStatisticsEnabled())
@@ -616,7 +616,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
             assert key != null;
 
             byte[] bytes = swapMgr.read(spaceName,
-                new SwapKey(key.value(cctx.cacheObjectContext(), false), part, keyBytes),
+                new SwapKey(key, part, keyBytes),
                 cctx.deploy().globalLoader());
 
             if (bytes == null && lsnr != null)
@@ -667,7 +667,8 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                         true,
                         null,
                         null,
-                        null);
+                        null,
+                        false);
 
                 return entry;
             }
@@ -690,7 +691,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
         final GridTuple<GridCacheSwapEntry> t = F.t1();
         final GridTuple<IgniteCheckedException> err = F.t1();
 
-        SwapKey swapKey = new SwapKey(key.value(cctx.cacheObjectContext(), false),
+        SwapKey swapKey = new SwapKey(key,
             part,
             key.valueBytes(cctx.cacheObjectContext()));
 
@@ -731,7 +732,8 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                                 true,
                                 null,
                                 null,
-                                null);
+                                null,
+                                false);
                         }
 
                         if (cctx.config().isStatisticsEnabled())
@@ -916,7 +918,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
                     if (cctx.events().isRecordable(EVT_CACHE_OBJECT_FROM_OFFHEAP))
                         cctx.events().addEvent(part, key, cctx.nodeId(), (IgniteUuid)null, null,
-                            EVT_CACHE_OBJECT_FROM_OFFHEAP, null, false, null, true, null, null, null);
+                            EVT_CACHE_OBJECT_FROM_OFFHEAP, null, false, null, true, null, null, null, false);
 
                     GridCacheBatchSwapEntry unswapped = new GridCacheBatchSwapEntry(key,
                         part,
@@ -938,7 +940,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                     if (unprocessedKeys == null)
                         unprocessedKeys = new ArrayList<>(keys.size());
 
-                    SwapKey swapKey = new SwapKey(key.value(cctx.cacheObjectContext(), false),
+                    SwapKey swapKey = new SwapKey(key,
                         cctx.affinity().partition(key),
                         key.valueBytes(cctx.cacheObjectContext()));
 
@@ -953,7 +955,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
             unprocessedKeys = new ArrayList<>(keys.size());
 
             for (KeyCacheObject key : keys) {
-                SwapKey swapKey = new SwapKey(key.value(cctx.cacheObjectContext(), false),
+                SwapKey swapKey = new SwapKey(key,
                     cctx.affinity().partition(key),
                     key.valueBytes(cctx.cacheObjectContext()));
 
@@ -1021,7 +1023,8 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                                     true,
                                     null,
                                     null,
-                                    null);
+                                    null,
+                                    false);
                             }
 
                             if (cctx.config().isStatisticsEnabled())
@@ -1220,7 +1223,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
         }
 
         if (swapEnabled) {
-            SwapKey swapKey = new SwapKey(key.value(cctx.cacheObjectContext(), false),
+            SwapKey swapKey = new SwapKey(key,
                 part,
                 key.valueBytes(cctx.cacheObjectContext()));
 
@@ -1286,7 +1289,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
             if (cctx.events().isRecordable(EVT_CACHE_OBJECT_TO_OFFHEAP))
                 cctx.events().addEvent(part, key, cctx.nodeId(), (IgniteUuid)null, null,
-                    EVT_CACHE_OBJECT_TO_OFFHEAP, null, false, null, true, null, null, null);
+                    EVT_CACHE_OBJECT_TO_OFFHEAP, null, false, null, true, null, null, null, false);
         }
         else if (swapEnabled)
             writeToSwap(part, key, entry.marshal());
@@ -1320,7 +1323,8 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
                 if (cctx.events().isRecordable(EVT_CACHE_OBJECT_TO_OFFHEAP))
                     cctx.events().addEvent(swapEntry.partition(), swapEntry.key(), cctx.nodeId(),
-                        (IgniteUuid)null, null, EVT_CACHE_OBJECT_TO_OFFHEAP, null, false, null, true, null, null, null);
+                        (IgniteUuid)null, null, EVT_CACHE_OBJECT_TO_OFFHEAP, null, false, null, true, null, null, null,
+                        false);
 
                 if (qryMgr.enabled())
                     qryMgr.onSwap(swapEntry.key());
@@ -1330,7 +1334,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
             Map<SwapKey, byte[]> batch = new LinkedHashMap<>();
 
             for (GridCacheBatchSwapEntry entry : swapped) {
-                SwapKey swapKey = new SwapKey(entry.key().value(cctx.cacheObjectContext(), false),
+                SwapKey swapKey = new SwapKey(entry.key(),
                     entry.partition(),
                     entry.key().valueBytes(cctx.cacheObjectContext()));
 
@@ -1342,7 +1346,8 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
             if (cctx.events().isRecordable(EVT_CACHE_OBJECT_SWAPPED)) {
                 for (GridCacheBatchSwapEntry batchSwapEntry : swapped) {
                     cctx.events().addEvent(batchSwapEntry.partition(), batchSwapEntry.key(), cctx.nodeId(),
-                        (IgniteUuid)null, null, EVT_CACHE_OBJECT_SWAPPED, null, false, null, true, null, null, null);
+                        (IgniteUuid)null, null, EVT_CACHE_OBJECT_SWAPPED, null, false, null, true, null, null, null,
+                        false);
 
                     if (qryMgr.enabled())
                         qryMgr.onSwap(batchSwapEntry.key());
@@ -1368,7 +1373,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
         checkIteratorQueue();
 
         swapMgr.write(spaceName,
-            new SwapKey(key.value(cctx.cacheObjectContext(), false), part, key.valueBytes(cctx.cacheObjectContext())),
+            new SwapKey(key, part, key.valueBytes(cctx.cacheObjectContext())),
             entry, cctx.deploy().globalLoader());
 
         if (cctx.config().isStatisticsEnabled())
@@ -1376,7 +1381,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         if (cctx.events().isRecordable(EVT_CACHE_OBJECT_SWAPPED))
             cctx.events().addEvent(part, key, cctx.nodeId(), (IgniteUuid) null, null,
-                EVT_CACHE_OBJECT_SWAPPED, null, false, null, true, null, null, null);
+                EVT_CACHE_OBJECT_SWAPPED, null, false, null, true, null, null, null, false);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index f7d115f..09ae714 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -148,6 +148,9 @@ public class GridCacheUtils {
     /** Skip store flag bit mask. */
     public static final int SKIP_STORE_FLAG_MASK = 0x1;
 
+    /** Keep serialized flag. */
+    public static final int KEEP_BINARY_FLAG_MASK = 0x2;
+
     /** Empty predicate array. */
     private static final IgnitePredicate[] EMPTY = new IgnitePredicate[0];
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index f39084e..b64ad9a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -311,7 +311,7 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public <K1, V1> IgniteCache<K1, V1> withKeepPortable() {
+    @Override public <K1, V1> IgniteCache<K1, V1> withKeepBinary() {
         return keepPortable();
     }
 
@@ -445,7 +445,7 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
         final CacheQuery<Map.Entry<K,V>> qry;
         final CacheQueryFuture<Map.Entry<K,V>> fut;
 
-        boolean isKeepPortable = opCtx != null && opCtx.isKeepPortable();
+        boolean isKeepPortable = opCtx != null && opCtx.isKeepBinary();
 
         if (filter instanceof ScanQuery) {
             IgniteBiPredicate<K, V> p = ((ScanQuery)filter).getFilter();
@@ -1672,7 +1672,7 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
      * </ul> <p> For example, if you use {@link Integer} as a key and {@code Value} class as a value (which will be
      * stored in portable format), you should acquire following projection to avoid deserialization:
      * <pre>
-     * IgniteInternalCache<Integer, GridPortableObject> prj = cache.keepPortable();
+     * IgniteInternalCache<Integer, GridPortableObject> prj = cache.keepBinary();
      *
      * // Value is not deserialized and returned in portable format.
      * GridPortableObject po = prj.get(1);
@@ -1726,7 +1726,7 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
             CacheOperationContext opCtx0 =
                 new CacheOperationContext(true,
                     opCtx != null ? opCtx.subjectId() : null,
-                    opCtx != null && opCtx.isKeepPortable(),
+                    opCtx != null && opCtx.isKeepBinary(),
                     opCtx != null ? opCtx.expiry() : null,
                     opCtx != null && opCtx.noRetries());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
index 167cc8e..4155706 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
@@ -188,7 +188,7 @@ import org.jetbrains.annotations.Nullable;
  * needed for performance reasons. To work with portable format directly you should create special projection
  * using {@link #keepPortable()} method:
  * <pre>
- * IgniteInternalCache<Integer, GridPortableObject> prj = Ignition.grid().cache(null).keepPortable();
+ * IgniteInternalCache<Integer, GridPortableObject> prj = Ignition.grid().cache(null).keepBinary();
  *
  * // Value is not deserialized and returned in portable format.
  * GridPortableObject po = prj.get(1);
@@ -231,7 +231,7 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K, V>> {
      * so keys and values will be returned from cache API methods without changes. Therefore,
      * signature of the projection can contain only following types:
      * <ul>
-     *     <li><code>org.gridgain.grid.portables.PortableObject</code> for portable classes</li>
+     *     <li><code>org.gridgain.grid.binary.PortableObject</code> for portable classes</li>
      *     <li>All primitives (byte, int, ...) and there boxed versions (Byte, Integer, ...)</li>
      *     <li>Arrays of primitives (byte[], int[], ...)</li>
      *     <li>{@link String} and array of {@link String}s</li>
@@ -249,7 +249,7 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K, V>> {
      * (which will be stored in portable format), you should acquire following projection
      * to avoid deserialization:
      * <pre>
-     * IgniteInternalCache<Integer, GridPortableObject> prj = cache.keepPortable();
+     * IgniteInternalCache<Integer, GridPortableObject> prj = cache.keepBinary();
      *
      * // Value is not deserialized and returned in portable format.
      * GridPortableObject po = prj.get(1);

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObjectImpl.java
index 6958c87..4db7ec1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObjectImpl.java
@@ -61,6 +61,11 @@ public class KeyCacheObjectImpl extends CacheObjectAdapter implements KeyCacheOb
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isPlatformType() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Nullable @Override public <T> T value(CacheObjectContext ctx, boolean cpy) {
         assert val != null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
index 1ff4575..edad586 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
@@ -312,8 +312,8 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
      * @param key Key.
      * @param rmv {@code True} if entry was removed.
      */
-    public void onEntryUpdated(KeyCacheObject key, boolean rmv) {
-        Object key0 = key.value(cctx.cacheObjectContext(), false);
+    public void onEntryUpdated(KeyCacheObject key, boolean rmv, boolean keepPortable) {
+        Object key0 = cctx.cacheObjectContext().unwrapPortableIfNeeded(key, keepPortable, false);
 
         if (key0 instanceof GridCacheSetItemKey)
             onSetItemUpdated((GridCacheSetItemKey)key0, rmv);

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
index 637e51b..1709b0f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
@@ -169,6 +169,8 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
 
             boolean skipStore = opCtx != null && opCtx.skipStore();
 
+            boolean keepBinary = opCtx != null && opCtx.isKeepBinary();
+
             do {
                 retry = false;
 
@@ -181,7 +183,7 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
                     ctx.kernalContext().task().setThreadContext(TC_SUBGRID, nodes);
 
                     retry = !ctx.kernalContext().task().execute(
-                        new RemoveAllTask(ctx.name(), topVer, skipStore), null).get();
+                        new RemoveAllTask(ctx.name(), topVer, skipStore, keepBinary), null).get();
                 }
             }
             while (ctx.affinity().affinityTopologyVersion().compareTo(topVer) != 0 || retry);
@@ -200,9 +202,7 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
 
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
-        boolean skipStore = opCtx != null && opCtx.skipStore();
-
-        removeAllAsync(opFut, topVer, skipStore);
+        removeAllAsync(opFut, topVer, opCtx != null && opCtx.skipStore(), opCtx != null && opCtx.isKeepBinary());
 
         return opFut;
     }
@@ -212,15 +212,19 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
      * @param topVer Topology version.
      * @param skipStore Skip store flag.
      */
-    private void removeAllAsync(final GridFutureAdapter<Void> opFut, final AffinityTopologyVersion topVer,
-        final boolean skipStore) {
+    private void removeAllAsync(
+        final GridFutureAdapter<Void> opFut,
+        final AffinityTopologyVersion topVer,
+        final boolean skipStore,
+        final boolean keepBinary
+    ) {
         Collection<ClusterNode> nodes = ctx.grid().cluster().forDataNodes(name()).nodes();
 
         if (!nodes.isEmpty()) {
             ctx.kernalContext().task().setThreadContext(TC_SUBGRID, nodes);
 
             IgniteInternalFuture<Boolean> rmvAll = ctx.kernalContext().task().execute(
-                new RemoveAllTask(ctx.name(), topVer, skipStore), null);
+                new RemoveAllTask(ctx.name(), topVer, skipStore, keepBinary), null);
 
             rmvAll.listen(new IgniteInClosure<IgniteInternalFuture<Boolean>>() {
                 @Override public void apply(IgniteInternalFuture<Boolean> fut) {
@@ -232,7 +236,7 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
                         if (topVer0.equals(topVer) && !retry)
                             opFut.onDone();
                         else
-                            removeAllAsync(opFut, topVer0, skipStore);
+                            removeAllAsync(opFut, topVer0, skipStore, keepBinary);
                     }
                     catch (ClusterGroupEmptyCheckedException ignore) {
                         if (log.isDebugEnabled())
@@ -277,15 +281,19 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
         /** Skip store flag. */
         private final boolean skipStore;
 
+        /** Keep binary flag. */
+        private final boolean keepBinary;
+
         /**
          * @param cacheName Cache name.
          * @param topVer Affinity topology version.
          * @param skipStore Skip store flag.
          */
-        public RemoveAllTask(String cacheName, AffinityTopologyVersion topVer, boolean skipStore) {
+        public RemoveAllTask(String cacheName, AffinityTopologyVersion topVer, boolean skipStore, boolean keepBinary) {
             this.cacheName = cacheName;
             this.topVer = topVer;
             this.skipStore = skipStore;
+            this.keepBinary = keepBinary;
         }
 
         /** {@inheritDoc} */
@@ -294,7 +302,7 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
             Map<ComputeJob, ClusterNode> jobs = new HashMap();
 
             for (ClusterNode node : subgrid)
-                jobs.put(new GlobalRemoveAllJob(cacheName, topVer, skipStore), node);
+                jobs.put(new GlobalRemoveAllJob(cacheName, topVer, skipStore, keepBinary), node);
 
             return jobs;
         }
@@ -335,15 +343,24 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
         /** Skip store flag. */
         private final boolean skipStore;
 
+        /** Keep binary flag. */
+        private final boolean keepBinary;
+
         /**
          * @param cacheName Cache name.
          * @param topVer Topology version.
          * @param skipStore Skip store flag.
          */
-        private GlobalRemoveAllJob(String cacheName, @NotNull AffinityTopologyVersion topVer, boolean skipStore) {
+        private GlobalRemoveAllJob(
+            String cacheName,
+            @NotNull AffinityTopologyVersion topVer,
+            boolean skipStore,
+            boolean keepBinary
+        ) {
             super(cacheName, topVer);
 
             this.skipStore = skipStore;
+            this.keepBinary = keepBinary;
         }
 
         /** {@inheritDoc} */
@@ -376,6 +393,7 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
                     ((DataStreamerImpl) dataLdr).maxRemapCount(0);
 
                     dataLdr.skipStore(skipStore);
+                    dataLdr.keepBinary(keepBinary);
 
                     dataLdr.receiver(DataStreamerCacheUpdaters.<KeyCacheObject, Object>batched());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
index a138d30..eaf6a7d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
@@ -746,8 +746,9 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
     @Override public boolean tmLock(IgniteInternalTx tx,
         long timeout,
         @Nullable GridCacheVersion serOrder,
-        GridCacheVersion serReadVer)
-        throws GridCacheEntryRemovedException, GridDistributedLockCancelledException {
+        GridCacheVersion serReadVer,
+        boolean keepBinary
+    ) throws GridCacheEntryRemovedException, GridDistributedLockCancelledException {
         if (tx.local())
             // Null is returned if timeout is negative and there is other lock owner.
             return addLocal(
@@ -821,7 +822,7 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
 
                 // Event notification.
                 cctx.events().addEvent(partition(), key, prev.nodeId(), prev, EVT_CACHE_OBJECT_UNLOCKED, val, hasVal,
-                    val, hasVal, null, null, null);
+                    val, hasVal, null, null, null, true);
             }
 
             if (owner != null && cctx.events().isRecordable(EVT_CACHE_OBJECT_LOCKED)) {
@@ -829,7 +830,7 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
 
                 // Event notification.
                 cctx.events().addEvent(partition(), key, owner.nodeId(), owner, EVT_CACHE_OBJECT_LOCKED, val, hasVal,
-                    val, hasVal, null, null, null);
+                    val, hasVal, null, null, null, true);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
index 2899e25..4c504ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
@@ -39,6 +39,7 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.apache.ignite.transactions.TransactionIsolation;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.internal.processors.cache.GridCacheUtils.KEEP_BINARY_FLAG_MASK;
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.SKIP_STORE_FLAG_MASK;
 
 /**
@@ -135,6 +136,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
         int keyCnt,
         int txSize,
         boolean skipStore,
+        boolean keepBinary,
         boolean addDepInfo
     ) {
         super(lockVer, keyCnt, addDepInfo);
@@ -158,6 +160,7 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
         retVals = new boolean[keyCnt];
 
         skipStore(skipStore);
+        keepBinary(keepBinary);
     }
 
     /**
@@ -243,6 +246,20 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
     }
 
     /**
+     * @param keepBinary Keep binary flag.
+     */
+    public void keepBinary(boolean keepBinary) {
+        flags = keepBinary ? (byte)(flags | KEEP_BINARY_FLAG_MASK) : (byte)(flags & ~KEEP_BINARY_FLAG_MASK);
+    }
+
+    /**
+     * @return Keep portable.
+     */
+    public boolean keepBinary() {
+        return (flags & KEEP_BINARY_FLAG_MASK) != 0;
+    }
+
+    /**
      * @return Transaction isolation or <tt>null</tt> if not in transaction.
      */
     public TransactionIsolation isolation() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
index fcbf58d..3aa7e1c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
@@ -586,6 +586,7 @@ public class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
                                                 false,
                                                 true,
                                                 true,
+                                                txEntry.keepBinary(),
                                                 topVer,
                                                 null,
                                                 replicate ? DR_BACKUP : DR_NONE,
@@ -602,6 +603,7 @@ public class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
                                                 txEntry.ttl(),
                                                 true,
                                                 true,
+                                                txEntry.keepBinary(),
                                                 topVer,
                                                 null,
                                                 replicate ? DR_BACKUP : DR_NONE,
@@ -632,6 +634,7 @@ public class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
                                             false,
                                             true,
                                             true,
+                                            txEntry.keepBinary(),
                                             topVer,
                                             null,
                                             replicate ? DR_BACKUP : DR_NONE,

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
index 1b2d834..60410c4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
@@ -263,8 +263,9 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry {
     @Override public boolean tmLock(IgniteInternalTx tx,
         long timeout,
         @Nullable GridCacheVersion serOrder,
-        GridCacheVersion serReadVer)
-        throws GridCacheEntryRemovedException, GridDistributedLockCancelledException {
+        GridCacheVersion serReadVer,
+        boolean keepBinary
+    ) throws GridCacheEntryRemovedException, GridDistributedLockCancelledException {
         if (tx.local()) {
             GridDhtTxLocalAdapter dhtTx = (GridDhtTxLocalAdapter)tx;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index 749d06a..c83e78c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -666,7 +666,8 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
                                     cached.hasValue(),
                                     null,
                                     null,
-                                    null);
+                                    null,
+                                    false);
                             }
                         }
                     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
index c175b0b..6ed8261 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
@@ -165,6 +165,9 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
     /** Skip store flag. */
     private final boolean skipStore;
 
+    /** Keep binary. */
+    private final boolean keepBinary;
+
     /**
      * @param cctx Cache context.
      * @param nearNodeId Near node ID.
@@ -193,7 +196,8 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
         long threadId,
         long accessTtl,
         CacheEntryPredicate[] filter,
-        boolean skipStore) {
+        boolean skipStore,
+        boolean keepBinary) {
         super(cctx.kernalContext(), CU.boolReducer());
 
         assert nearNodeId != null;
@@ -211,6 +215,7 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
         this.tx = tx;
         this.accessTtl = accessTtl;
         this.skipStore = skipStore;
+        this.keepBinary = keepBinary;
 
         if (tx != null)
             tx.topologyVersion(topVer);
@@ -870,6 +875,7 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
                         inTx() ? tx.taskNameHash() : 0,
                         read ? accessTtl : -1L,
                         skipStore,
+                        keepBinary,
                         cctx.deploymentEnabled());
 
                     try {
@@ -1177,7 +1183,7 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
                             if (rec && !entry.isInternal())
                                 cctx.events().addEvent(entry.partition(), entry.key(), cctx.localNodeId(),
                                     (IgniteUuid)null, null, EVT_CACHE_REBALANCE_OBJECT_LOADED, info.value(), true, null,
-                                    false, null, null, null);
+                                    false, null, null, null, false);
                         }
                     }
                     catch (IgniteCheckedException e) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java
index 91ab1ca..efcbdc6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java
@@ -120,6 +120,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
      * @param taskNameHash Task name hash code.
      * @param accessTtl TTL for read operation.
      * @param skipStore Skip store flag.
+     * @param keepBinary Keep binary flag.
      * @param addDepInfo Deployment info flag.
      */
     public GridDhtLockRequest(
@@ -143,6 +144,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
         int taskNameHash,
         long accessTtl,
         boolean skipStore,
+        boolean keepBinary,
         boolean addDepInfo
     ) {
         super(cacheId,
@@ -159,6 +161,7 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
             dhtCnt == 0 ? nearCnt : dhtCnt,
             txSize,
             skipStore,
+            keepBinary,
             addDepInfo);
 
         this.topVer = topVer;

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
index 4ce4759..732ff43 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
@@ -39,7 +39,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
-import org.apache.ignite.internal.processors.cache.GridCacheFilterFailedException;
 import org.apache.ignite.internal.processors.cache.GridCacheLockTimeoutException;
 import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
 import org.apache.ignite.internal.processors.cache.GridCacheReturn;
@@ -62,7 +61,6 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.util.F0;
 import org.apache.ignite.internal.util.GridLeanSet;
-import org.apache.ignite.internal.util.future.GridEmbeddedFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.lang.GridClosureException;
 import org.apache.ignite.internal.util.typedef.C1;
@@ -256,7 +254,8 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
                                 null,
                                 null,
                                 req.accessTtl(),
-                                req.skipStore());
+                                req.skipStore(),
+                                req.keepBinary());
                         }
 
                         entry = entryExx(key, req.topologyVersion());
@@ -604,7 +603,8 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
             isolation,
             accessTtl,
             CU.empty0(),
-            opCtx != null && opCtx.skipStore());
+            opCtx != null && opCtx.skipStore(),
+            opCtx != null && opCtx.isKeepBinary());
     }
 
     /**
@@ -631,7 +631,8 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
         TransactionIsolation isolation,
         long accessTtl,
         CacheEntryPredicate[] filter,
-        boolean skipStore) {
+        boolean skipStore,
+        boolean keepBinary) {
         if (keys == null || keys.isEmpty())
             return new GridDhtFinishedFuture<>(true);
 
@@ -652,7 +653,8 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
             tx.threadId(),
             accessTtl,
             filter,
-            skipStore);
+            skipStore,
+            keepBinary);
 
         for (KeyCacheObject key : keys) {
             try {
@@ -772,7 +774,8 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
                         req.threadId(),
                         req.accessTtl(),
                         filter,
-                        req.skipStore());
+                        req.skipStore(),
+                        req.keepBinary());
 
                     // Add before mapping.
                     if (!ctx.mvcc().addFuture(fut))
@@ -916,7 +919,8 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
                     req.txRead(),
                     req.needReturnValue(),
                     req.accessTtl(),
-                    req.skipStore());
+                    req.skipStore(),
+                    req.keepBinary());
 
                 final GridDhtTxLocal t = tx;
 
@@ -1124,7 +1128,8 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
                                         CU.subjectId(tx, ctx.shared()),
                                         null,
                                         tx != null ? tx.resolveTaskName() : null,
-                                        null);
+                                        null,
+                                        req.keepBinary());
 
                                 assert e.lockedBy(mappedVer) ||
                                     (ctx.mvcc().isRemoved(e.context(), mappedVer) && req.timeout() > 0) :

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
index 8c7d985..0cb5853 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
@@ -153,7 +153,7 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
             invalidate,
             storeEnabled,
             onePhaseCommit,
-            txSize, 
+            txSize,
             subjId, 
             taskNameHash
         );
@@ -613,7 +613,8 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
         final boolean read,
         final boolean needRetVal,
         long accessTtl,
-        boolean skipStore
+        boolean skipStore,
+        boolean keepBinary
     ) {
         try {
             checkValid();
@@ -681,7 +682,8 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
                         -1L,
                         -1L,
                         null,
-                        skipStore);
+                        skipStore,
+                        keepBinary);
 
                     if (read)
                         txEntry.ttl(accessTtl);
@@ -719,7 +721,8 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
                 skipped,
                 accessTtl,
                 null,
-                skipStore);
+                skipStore,
+                keepBinary);
         }
         catch (IgniteCheckedException e) {
             setRollbackOnly();
@@ -749,7 +752,8 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
         final Set<KeyCacheObject> skipped,
         final long accessTtl,
         @Nullable final CacheEntryPredicate[] filter,
-        boolean skipStore) {
+        boolean skipStore,
+        boolean keepBinary) {
         if (log.isDebugEnabled())
             log.debug("Before acquiring transaction lock on keys [passedKeys=" + passedKeys + ", skipped=" +
                 skipped + ']');
@@ -768,7 +772,8 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
             isolation,
             accessTtl,
             CU.empty0(),
-            skipStore);
+            skipStore,
+            keepBinary);
 
         return new GridEmbeddedFuture<>(
             fut,