You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2016/01/21 08:33:07 UTC

[19/19] ignite git commit: Pass strongly typed cache objects to the cache store manager

Pass strongly typed cache objects to the cache store manager


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

Branch: refs/heads/sql-store
Commit: 20217fae3bd2fc20ec68c84ffafe31a75ee1987c
Parents: 965846c
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Jan 21 10:32:19 2016 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Jan 21 10:32:19 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheMapEntry.java     | 16 +++----
 .../distributed/dht/GridDhtCacheEntry.java      |  2 +-
 .../distributed/dht/GridDhtLocalPartition.java  |  2 +-
 .../dht/atomic/GridDhtAtomicCache.java          | 13 +++---
 .../local/atomic/GridLocalAtomicCache.java      | 30 +++++++------
 .../cache/store/CacheStoreManager.java          | 15 ++++---
 .../store/GridCacheStoreManagerAdapter.java     | 44 +++++++++++---------
 .../transactions/IgniteTxLocalAdapter.java      |  6 +--
 8 files changed, 72 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/20217fae/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 c896a82..0cd4fd4 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
@@ -1220,7 +1220,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         // Persist outside of synchronization. The correctness of the
         // value will be handled by current transaction.
         if (writeThrough)
-            cctx.store().put(tx, keyValue(false), CU.value(val, cctx, false), newVer);
+            cctx.store().put(tx, key, val, newVer);
 
         if (intercept)
             cctx.config().getInterceptor().onAfterPut(new CacheLazyEntry(cctx, key, key0, val, val0, keepBinary));
@@ -1654,7 +1654,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 if (writeThrough)
                     // Must persist inside synchronization in non-tx mode.
-                    cctx.store().put(null, keyValue(false), CU.value(updated, cctx, false), ver);
+                    cctx.store().put(null, key, updated, ver);
 
                 // Update index inside synchronization since it can be updated
                 // in load methods without actually holding entry lock.
@@ -1688,7 +1688,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             else {
                 if (writeThrough)
                     // Must persist inside synchronization in non-tx mode.
-                    cctx.store().remove(null, keyValue(false));
+                    cctx.store().remove(null, key);
 
                 boolean hasValPtr = hasOffHeapPointer();
 
@@ -1905,10 +1905,10 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                             if (val == null) {
                                 assert deletedUnlocked();
 
-                                cctx.store().remove(null, keyValue(false));
+                                cctx.store().remove(null, key);
                             }
                             else
-                                cctx.store().put(null, keyValue(false), CU.value(val, cctx, false), ver);
+                                cctx.store().put(null, key, val, ver);
                         }
 
                         return new GridCacheUpdateAtomicResult(false,
@@ -1958,10 +1958,10 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                             if (val == null) {
                                 assert deletedUnlocked();
 
-                                cctx.store().remove(null, keyValue(false));
+                                cctx.store().remove(null, key);
                             }
                             else
-                                cctx.store().put(null, keyValue(false), CU.value(val, cctx, false), ver);
+                                cctx.store().put(null, key, val, ver);
                         }
                         else {
                             if (log.isDebugEnabled())
@@ -3244,7 +3244,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 if (cctx.store().isLocal()) {
                     if (val != null)
-                        cctx.store().put(null, keyValue(false), CU.value(val, cctx, false), ver);
+                        cctx.store().put(null, key, val, ver);
                 }
 
                 return true;

http://git-wip-us.apache.org/repos/asf/ignite/blob/20217fae/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 14e3d3e..7ff5bdb 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
@@ -590,7 +590,7 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry {
                 }
 
                 if (cctx.store().isLocal())
-                    cctx.store().remove(null, keyValue(false));
+                    cctx.store().remove(null, key);
 
                 rmv = true;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/20217fae/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 c4312b5..409cc98 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
@@ -572,7 +572,7 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
                     cctx.swap().remove(key);
 
                     if (isLocStore)
-                        cctx.store().remove(null, key.value(cctx.cacheObjectContext(), false));
+                        cctx.store().remove(null, key);
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/20217fae/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 81fd5d6..c02eb95 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -2145,11 +2145,14 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                     putMap;
 
                 try {
-                    ctx.store().putAll(null, F.viewReadOnly(storeMap, new C1<CacheObject, IgniteBiTuple<CacheObject, GridCacheVersion>>() {
-                        @Override public IgniteBiTuple<CacheObject, GridCacheVersion> apply(CacheObject v) {
-                            return F.t(v, ver);
-                        }
-                    }));
+                    Map<? extends KeyCacheObject, IgniteBiTuple<? extends CacheObject, GridCacheVersion>> view = F.viewReadOnly(storeMap,
+                        new C1<CacheObject, IgniteBiTuple<? extends CacheObject, GridCacheVersion>>() {
+                            @Override public IgniteBiTuple<? extends CacheObject, GridCacheVersion> apply(CacheObject val) {
+                                return F.t(val, ver);
+                            }
+                        });
+
+                    ctx.store().putAll(null, view);
                 }
                 catch (CacheStorePartialUpdateException e) {
                     storeErr = e;

http://git-wip-us.apache.org/repos/asf/ignite/blob/20217fae/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
index 6130ead..d9190c5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -73,6 +73,7 @@ import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiClosure;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.plugin.security.SecurityPermission;
 import org.apache.ignite.transactions.TransactionIsolation;
@@ -1084,9 +1085,9 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
         try {
             int size = locked.size();
 
-            Map<Object, Object> putMap = null;
+            Map<KeyCacheObject, CacheObject> putMap = null;
 
-            Collection<Object> rmvKeys = null;
+            Collection<KeyCacheObject> rmvKeys = null;
 
             List<CacheObject> writeVals = null;
 
@@ -1208,7 +1209,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
                             if (rmvKeys == null)
                                 rmvKeys = new ArrayList<>(size);
 
-                            rmvKeys.add(entry.key().value(ctx.cacheObjectContext(), false));
+                            rmvKeys.add(entry.key());
                         }
                         else {
                             if (intercept) {
@@ -1246,7 +1247,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
                                 writeVals = new ArrayList<>(size);
                             }
 
-                            putMap.put(CU.value(entry.key(), ctx, false), CU.value(updated, ctx, false));
+                            putMap.put(entry.key(), updated);
                             writeVals.add(updated);
                         }
                     }
@@ -1282,7 +1283,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
                             writeVals = new ArrayList<>(size);
                         }
 
-                        putMap.put(CU.value(entry.key(), ctx, false), CU.value(cacheVal, ctx, false));
+                        putMap.put(entry.key(), cacheVal);
                         writeVals.add(cacheVal);
                     }
                     else {
@@ -1313,7 +1314,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
                         if (rmvKeys == null)
                             rmvKeys = new ArrayList<>(size);
 
-                        rmvKeys.add(entry.key().value(ctx.cacheObjectContext(), false));
+                        rmvKeys.add(entry.key());
                     }
 
                     filtered.add(entry);
@@ -1373,8 +1374,8 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
         List<GridCacheEntryEx> entries,
         final GridCacheVersion ver,
         @Nullable List<CacheObject> writeVals,
-        @Nullable Map<Object, Object> putMap,
-        @Nullable Collection<Object> rmvKeys,
+        @Nullable Map<KeyCacheObject, CacheObject> putMap,
+        @Nullable Collection<KeyCacheObject> rmvKeys,
         @Nullable ExpiryPolicy expiryPlc,
         boolean keepBinary,
         @Nullable CachePartialUpdateCheckedException err,
@@ -1389,11 +1390,14 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
         try {
             if (putMap != null) {
                 try {
-                    ctx.store().putAll(null, F.viewReadOnly(putMap, new C1<Object, IgniteBiTuple<Object, GridCacheVersion>>() {
-                        @Override public IgniteBiTuple<Object, GridCacheVersion> apply(Object v) {
-                            return F.t(v, ver);
-                        }
-                    }));
+                    Map<? extends KeyCacheObject, IgniteBiTuple<? extends CacheObject, GridCacheVersion>> view = F.viewReadOnly(putMap,
+                        new C1<CacheObject, IgniteBiTuple<? extends CacheObject, GridCacheVersion>>() {
+                            @Override public IgniteBiTuple<? extends CacheObject, GridCacheVersion> apply(CacheObject val) {
+                                return F.t(val, ver);
+                            }
+                        });
+
+                    ctx.store().putAll(null, view);
                 }
                 catch (CacheStorePartialUpdateException e) {
                     storeErr = e;

http://git-wip-us.apache.org/repos/asf/ignite/blob/20217fae/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
index 67c9334..16fbf9e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
@@ -21,6 +21,7 @@ import java.util.Collection;
 import java.util.Map;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheManager;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
@@ -33,7 +34,7 @@ import org.jetbrains.annotations.Nullable;
 /**
  * Cache store manager interface.
  */
-public interface CacheStoreManager<K, V> extends GridCacheManager<K, V> {
+public interface CacheStoreManager extends GridCacheManager {
     /**
      * Initialize store manager.
      *
@@ -131,7 +132,7 @@ public interface CacheStoreManager<K, V> extends GridCacheManager<K, V> {
      * @return {@code true} If there is a persistent storage.
      * @throws IgniteCheckedException If storage failed.
      */
-    public boolean put(@Nullable IgniteInternalTx tx, Object key, Object val, GridCacheVersion ver)
+    public boolean put(@Nullable IgniteInternalTx tx, KeyCacheObject key, CacheObject val, GridCacheVersion ver)
         throws IgniteCheckedException;
 
     /**
@@ -142,8 +143,10 @@ public interface CacheStoreManager<K, V> extends GridCacheManager<K, V> {
      * @return {@code True} if there is a persistent storage.
      * @throws IgniteCheckedException If storage failed.
      */
-    public boolean putAll(@Nullable IgniteInternalTx tx, Map<Object, IgniteBiTuple<Object, GridCacheVersion>> map)
-        throws IgniteCheckedException;
+    public boolean putAll(
+        @Nullable IgniteInternalTx tx,
+        Map<? extends KeyCacheObject, IgniteBiTuple<? extends CacheObject, GridCacheVersion>> map
+    ) throws IgniteCheckedException;
 
     /**
      * @param tx Cache transaction.
@@ -151,7 +154,7 @@ public interface CacheStoreManager<K, V> extends GridCacheManager<K, V> {
      * @return {@code True} if there is a persistent storage.
      * @throws IgniteCheckedException If storage failed.
      */
-    public boolean remove(@Nullable IgniteInternalTx tx, Object key) throws IgniteCheckedException;
+    public boolean remove(@Nullable IgniteInternalTx tx, KeyCacheObject key) throws IgniteCheckedException;
 
     /**
      * @param tx Cache transaction.
@@ -159,7 +162,7 @@ public interface CacheStoreManager<K, V> extends GridCacheManager<K, V> {
      * @return {@code True} if there is a persistent storage.
      * @throws IgniteCheckedException If storage failed.
      */
-    public boolean removeAll(@Nullable IgniteInternalTx tx, Collection<Object> keys)
+    public boolean removeAll(@Nullable IgniteInternalTx tx, Collection<? extends KeyCacheObject> keys)
         throws IgniteCheckedException;
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/20217fae/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index 7ffebbd..f1cfd94 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -37,6 +37,7 @@ import org.apache.ignite.cache.store.CacheStoreSessionListener;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
+import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheStoreBalancingWrapper;
 import org.apache.ignite.internal.processors.cache.CacheStorePartialUpdateException;
 import org.apache.ignite.internal.processors.cache.GridCacheInternal;
@@ -60,7 +61,6 @@ import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiInClosure;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lifecycle.LifecycleAware;
 import org.apache.ignite.transactions.Transaction;
 import org.jetbrains.annotations.NotNull;
@@ -526,25 +526,25 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean put(@Nullable IgniteInternalTx tx, Object key, Object val, GridCacheVersion ver)
+    @Override public boolean put(@Nullable IgniteInternalTx tx, KeyCacheObject key, CacheObject val, GridCacheVersion ver)
         throws IgniteCheckedException {
         if (store != null) {
             // Never persist internal keys.
             if (key instanceof GridCacheInternal)
                 return true;
 
-            key = cctx.unwrapBinaryIfNeeded(key, !convertBinary());
-            val = cctx.unwrapBinaryIfNeeded(val, !convertBinary());
+            Object key0 = cctx.unwrapBinaryIfNeeded(key, !convertBinary());
+            Object val0 = cctx.unwrapBinaryIfNeeded(val, !convertBinary());
 
             if (log.isDebugEnabled())
-                log.debug("Storing value in cache store [key=" + key + ", val=" + val + ']');
+                log.debug("Storing value in cache store [key=" + key0 + ", val=" + val0 + ']');
 
             sessionInit0(tx);
 
             boolean threwEx = true;
 
             try {
-                store.write(new CacheEntryImpl<>(key, locStore ? F.t(val, ver) : val));
+                store.write(new CacheEntryImpl<>(key0, locStore ? F.t(val0, ver) : val0));
 
                 threwEx = false;
             }
@@ -562,7 +562,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
             }
 
             if (log.isDebugEnabled())
-                log.debug("Stored value in cache store [key=" + key + ", val=" + val + ']');
+                log.debug("Stored value in cache store [key=" + key0 + ", val=" + val0 + ']');
 
             return true;
         }
@@ -571,13 +571,16 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean putAll(@Nullable IgniteInternalTx tx, Map map) throws IgniteCheckedException {
+    @Override public boolean putAll(
+        @Nullable IgniteInternalTx tx,
+        Map<? extends KeyCacheObject, IgniteBiTuple<? extends CacheObject, GridCacheVersion>> map
+    ) throws IgniteCheckedException {
         if (F.isEmpty(map))
             return true;
 
         if (map.size() == 1) {
-            Map.Entry<Object, IgniteBiTuple<Object, GridCacheVersion>> e =
-                ((Map<Object, IgniteBiTuple<Object, GridCacheVersion>>)map).entrySet().iterator().next();
+            Map.Entry<? extends KeyCacheObject, IgniteBiTuple<? extends CacheObject, GridCacheVersion>> e =
+                ((Map<? extends KeyCacheObject, IgniteBiTuple<? extends CacheObject, GridCacheVersion>>)map).entrySet().iterator().next();
 
             return put(tx, e.getKey(), e.getValue().get1(), e.getValue().get2());
         }
@@ -630,23 +633,23 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean remove(@Nullable IgniteInternalTx tx, Object key) throws IgniteCheckedException {
+    @Override public boolean remove(@Nullable IgniteInternalTx tx, KeyCacheObject key) throws IgniteCheckedException {
         if (store != null) {
             // Never remove internal key from store as it is never persisted.
             if (key instanceof GridCacheInternal)
                 return false;
 
-            key = cctx.unwrapBinaryIfNeeded(key, !convertBinary());
+            Object key0 = cctx.unwrapBinaryIfNeeded(key, !convertBinary());
 
             if (log.isDebugEnabled())
-                log.debug("Removing value from cache store [key=" + key + ']');
+                log.debug("Removing value from cache store [key=" + key0 + ']');
 
             sessionInit0(tx);
 
             boolean threwEx = true;
 
             try {
-                store.delete(key);
+                store.delete(key0);
 
                 threwEx = false;
             }
@@ -664,7 +667,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
             }
 
             if (log.isDebugEnabled())
-                log.debug("Removed value from cache store [key=" + key + ']');
+                log.debug("Removed value from cache store [key=" + key0 + ']');
 
             return true;
         }
@@ -673,12 +676,15 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean removeAll(@Nullable IgniteInternalTx tx, Collection keys) throws IgniteCheckedException {
+    @Override public boolean removeAll(
+        @Nullable IgniteInternalTx tx,
+        Collection<? extends KeyCacheObject> keys
+    ) throws IgniteCheckedException {
         if (F.isEmpty(keys))
             return true;
 
         if (keys.size() == 1) {
-            Object key = keys.iterator().next();
+            KeyCacheObject key = keys.iterator().next();
 
             return remove(tx, key);
         }
@@ -1024,7 +1030,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
      */
     private class EntriesView extends AbstractCollection<Cache.Entry<?, ?>> {
         /** */
-        private final Map<?, IgniteBiTuple<?, GridCacheVersion>> map;
+        private final Map<? extends KeyCacheObject, IgniteBiTuple<? extends CacheObject, GridCacheVersion>> map;
 
         /** */
         private Set<Object> rmvd;
@@ -1035,7 +1041,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
         /**
          * @param map Map.
          */
-        private EntriesView(Map<?, IgniteBiTuple<?, GridCacheVersion>> map) {
+        private EntriesView(Map<? extends KeyCacheObject, IgniteBiTuple<? extends CacheObject, GridCacheVersion>> map) {
             assert map != null;
 
             this.map = map;

http://git-wip-us.apache.org/repos/asf/ignite/blob/20217fae/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 926eaf2..430f08c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -658,8 +658,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
         if (near() || isWriteToStoreFromDht) {
             try {
                 if (writeEntries != null) {
-                    Map<Object, IgniteBiTuple<Object, GridCacheVersion>> putMap = null;
-                    List<Object> rmvCol = null;
+                    Map<KeyCacheObject, IgniteBiTuple<? extends CacheObject, GridCacheVersion>> putMap = null;
+                    List<KeyCacheObject> rmvCol = null;
                     CacheStoreManager writeStore = null;
 
                     boolean skipNonPrimary = near() && isWriteToStoreFromDht;
@@ -737,7 +737,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
                                 if (putMap == null)
                                     putMap = new LinkedHashMap<>(writeMap().size(), 1.0f);
 
-                                putMap.put(key, F.<Object, GridCacheVersion>t(val, ver));
+                                putMap.put(key, F.<CacheObject, GridCacheVersion>t(val, ver));
                             }
                         }
                         else if (op == DELETE) {