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 2017/04/12 07:33:46 UTC

[15/57] [abbrv] ignite git commit: IGNITE-4535 - Add option to store deserialized values on-heap

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 2564787..b81be12 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -342,11 +342,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         String msg = "Disable eviction policy (remove from configuration)";
 
-        if (cfg.getEvictionPolicy() != null) {
+        if (cfg.getEvictionPolicy() != null)
             perf.add(msg, false);
-
-            perf.add("Disable synchronized evictions (set 'evictSynchronized' to false)", !cfg.isEvictSynchronized());
-        }
         else
             perf.add(msg, true);
 
@@ -514,6 +511,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (cc.getAtomicityMode() == ATOMIC)
             assertParameter(cc.getTransactionManagerLookupClassName() == null,
                 "transaction manager can not be used with ATOMIC cache");
+
+        if (cc.getEvictionPolicy() != null && !cc.isOnheapCacheEnabled())
+            throw new IgniteCheckedException("Onheap cache must be enabled if eviction policy is configured [cacheName="
+                + U.maskName(cc.getName()) + "]");
     }
 
     /**
@@ -1476,7 +1477,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         GridCacheAffinityManager affMgr = new GridCacheAffinityManager();
         GridCacheEventManager evtMgr = new GridCacheEventManager();
-        CacheEvictionManager evictMgr = nearEnabled ? new GridCacheEvictionManager() : new CacheOffheapEvictionManager();
+        CacheEvictionManager evictMgr = (nearEnabled || cfg.isOnheapCacheEnabled()) ? new GridCacheEvictionManager() : new CacheOffheapEvictionManager();
         GridCacheQueryManager qryMgr = queryManager(cfg);
         CacheContinuousQueryManager contQryMgr = new CacheContinuousQueryManager();
         CacheDataStructuresManager dataStructuresMgr = new CacheDataStructuresManager();
@@ -1623,7 +1624,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
              * 7. GridCacheTtlManager.
              * ===============================================
              */
-            evictMgr = new CacheOffheapEvictionManager();
+            evictMgr = cfg.isOnheapCacheEnabled() ? new GridCacheEvictionManager() : new CacheOffheapEvictionManager();
             evtMgr = new GridCacheEventManager();
             pluginMgr = new CachePluginManager(ctx, cfg);
             drMgr = pluginMgr.createComponent(GridCacheDrManager.class);
@@ -3332,15 +3333,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 "Write behind flush thread count", locAttr.writeBehindFlushThreadCount(),
                 rmtAttr.writeBehindFlushThreadCount(), false);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "evictMaxOverflowRatio",
-                "Eviction max overflow ratio", locAttr.evictMaxOverflowRatio(),
-                rmtAttr.evictMaxOverflowRatio(), true);
-
             if (locAttr.cacheMode() == PARTITIONED) {
-                CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "evictSynchronized",
-                    "Eviction synchronized", locAttr.evictSynchronized(), rmtAttr.evictSynchronized(),
-                    true);
-
                 CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "nearEvictionPolicy",
                     "Near eviction policy", locAttr.nearEvictionPolicyClassName(),
                     rmtAttr.nearEvictionPolicyClassName(), false);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/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 823988c..457c895 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
@@ -920,14 +920,6 @@ public class GridCacheUtils {
     public static void unwindEvicts(GridCacheContext ctx) {
         assert ctx != null;
 
-        ctx.evicts().unwind();
-
-        if (ctx.isNear()) {
-            GridCacheContext dhtCtx = ctx.near().dht().context();
-
-            dhtCtx.evicts().unwind();
-        }
-
         ctx.ttl().expire();
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridNoStorageCacheMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridNoStorageCacheMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridNoStorageCacheMap.java
index d49b016..db3533d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridNoStorageCacheMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridNoStorageCacheMap.java
@@ -22,6 +22,7 @@ import java.util.Collections;
 import java.util.Set;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException;
 import org.jetbrains.annotations.Nullable;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
index 9b10d71..ae42784 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
@@ -250,7 +250,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
             for (GridDhtLocalPartition locPart : cctx.topology().currentLocalPartitions()) {
                 if (primary) {
                     if (cctx.affinity().primaryByPartition(locNode, locPart.id(), topVer)) {
-                        cnt += locPart.size();
+                        cnt += locPart.dataStore().size();
 
                         continue;
                     }
@@ -258,7 +258,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
 
                 if (backup) {
                     if (cctx.affinity().backupByPartition(locNode, locPart.id(), topVer))
-                        cnt += locPart.size();
+                        cnt += locPart.dataStore().size();
                 }
             }
 
@@ -276,7 +276,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         else {
             GridDhtLocalPartition locPart = cctx.topology().localPartition(part, AffinityTopologyVersion.NONE, false);
 
-            return locPart == null ? 0 : locPart.size();
+            return locPart == null ? 0 : locPart.dataStore().size();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/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 942462a..096ca9f 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
@@ -280,6 +280,12 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
 
             if (modes.offheap)
                 size += offheap.entriesCount(modes.primary, modes.backup, topVer);
+            else if (modes.heap) {
+                for (GridDhtLocalPartition locPart : ctx.topology().currentLocalPartitions()) {
+                    if ((modes.primary && locPart.primary(topVer)) || (modes.backup && locPart.backup(topVer)))
+                        size += locPart.publicSize();
+                }
+            }
         }
 
         return size;

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index ac6eee3..7e6ae81 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -398,16 +398,6 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
      *
      * @throws GridDhtInvalidPartitionException If partition for the key is no longer valid.
      */
-    @Override public GridCacheEntryEx entryEx(KeyCacheObject key, boolean touch)
-        throws GridDhtInvalidPartitionException {
-        return super.entryEx(key, touch);
-    }
-
-    /**
-     * {@inheritDoc}
-     *
-     * @throws GridDhtInvalidPartitionException If partition for the key is no longer valid.
-     */
     @Override public GridCacheEntryEx entryEx(KeyCacheObject key,
         AffinityTopologyVersion topVer) throws GridDhtInvalidPartitionException {
         return super.entryEx(key, topVer);
@@ -543,7 +533,7 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
 
                     CacheObject cacheVal = ctx.toCacheObject(val);
 
-                    entry = entryEx(key, false);
+                    entry = entryEx(key);
 
                     entry.initialValue(cacheVal,
                         ver,
@@ -579,6 +569,21 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
     }
 
     /** {@inheritDoc} */
+    @Override public int size() {
+        return (int)sizeLong();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long sizeLong() {
+        long sum = 0;
+
+        for (GridDhtLocalPartition p : topology().currentLocalPartitions())
+            sum += p.dataStore().size();
+
+        return sum;
+    }
+
+    /** {@inheritDoc} */
     @Override public int primarySize() {
         return (int)primarySizeLong();
     }
@@ -591,7 +596,7 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
 
         for (GridDhtLocalPartition p : topology().currentLocalPartitions()) {
             if (p.primary(topVer))
-                sum += p.publicSize();
+                sum += p.dataStore().size();
         }
 
         return sum;

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/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 e2c05cc..46b2bf8 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
@@ -78,7 +78,7 @@ import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDh
 /**
  * Key partition.
  */
-public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>, GridReservable, GridCacheConcurrentMap {
+public class GridDhtLocalPartition extends GridCacheConcurrentMapImpl implements Comparable<GridDhtLocalPartition>, GridReservable {
     /** Maximum size for delete queue. */
     public static final int MAX_DELETE_QUEUE_SIZE = Integer.getInteger(IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE, 200_000);
 
@@ -109,9 +109,6 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
     @GridToStringExclude
     private final GridFutureAdapter<?> rent;
 
-    /** Entries map. */
-    private final GridCacheConcurrentMap map;
-
     /** Context. */
     private final GridCacheContext cctx;
 
@@ -151,7 +148,7 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
      */
     @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor")
     GridDhtLocalPartition(GridCacheContext cctx, int id, GridCacheMapEntryFactory entryFactory) {
-        assert cctx != null;
+        super(cctx, entryFactory, cctx.config().getStartSize() / cctx.affinity().partitions());
 
         this.id = id;
         this.cctx = cctx;
@@ -160,12 +157,10 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
 
         rent = new GridFutureAdapter<Object>() {
             @Override public String toString() {
-                return "PartitionRentFuture [part=" + GridDhtLocalPartition.this + ", map=" + map + ']';
+                return "PartitionRentFuture [part=" + GridDhtLocalPartition.this + ']';
             }
         };
 
-        map = new GridCacheConcurrentMapImpl(cctx, entryFactory, cctx.config().getStartSize() / cctx.affinity().partitions());
-
         int delQueueSize = CU.isSystemCache(cctx.name()) ? 100 :
             Math.max(MAX_DELETE_QUEUE_SIZE / cctx.affinity().partitions(), 20);
 
@@ -240,20 +235,13 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
     }
 
     /**
-     * @return Keys belonging to partition.
-     */
-    public Set<KeyCacheObject> keySet() {
-        return map.keySet();
-    }
-
-    /**
      * @return {@code True} if partition is empty.
      */
     public boolean isEmpty() {
         if (cctx.allowFastEviction())
-            return map.size() == 0;
+            return size() == 0;
 
-        return size() == 0 && map.size() == 0;
+        return store.size() == 0 && size() == 0;
     }
 
     /**
@@ -294,26 +282,6 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
         }
     }
 
-    /** {@inheritDoc} */
-    @Override public int size() {
-        return (int)store.size();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int publicSize() {
-        return (int)store.size();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void incrementPublicSize(GridCacheEntryEx e) {
-        map.incrementPublicSize(e);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void decrementPublicSize(GridCacheEntryEx e) {
-        map.decrementPublicSize(e);
-    }
-
     /**
      * @return If partition is moving or owning or renting.
      */
@@ -323,52 +291,6 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
         return state == MOVING || state == OWNING || state == RENTING;
     }
 
-    /** {@inheritDoc} */
-    @Override @Nullable public GridCacheMapEntry getEntry(KeyCacheObject key) {
-        return map.getEntry(key);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean removeEntry(GridCacheEntryEx entry) {
-        return map.removeEntry(entry);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Iterable<GridCacheMapEntry> entries(
-        CacheEntryPredicate... filter) {
-        return map.entries(filter);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Iterable<GridCacheMapEntry> allEntries(CacheEntryPredicate... filter) {
-        return map.allEntries(filter);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Set<GridCacheMapEntry> entrySet(CacheEntryPredicate... filter) {
-        return map.entrySet(filter);
-    }
-
-    /** {@inheritDoc} */
-    @Override @Nullable public GridCacheMapEntry randomEntry() {
-        return map.randomEntry();
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridCacheMapEntry putEntryIfObsoleteOrAbsent(
-        AffinityTopologyVersion topVer,
-        KeyCacheObject key,
-        @Nullable CacheObject val,
-        boolean create,
-        boolean touch) {
-        return map.putEntryIfObsoleteOrAbsent(topVer, key, val, create, touch);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Set<KeyCacheObject> keySet(CacheEntryPredicate... filter) {
-        return map.keySet(filter);
-    }
-
     /**
      * @param entry Entry to remove.
      */
@@ -376,7 +298,7 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
         assert entry.obsolete() : entry;
 
         // Make sure to remove exactly this entry.
-        map.removeEntry(entry);
+        removeEntry(entry);
 
         // Attempt to evict.
         try {
@@ -895,20 +817,20 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
 
         boolean rec = cctx.events().isRecordable(EVT_CACHE_REBALANCE_OBJECT_UNLOADED);
 
-        Iterator<GridDhtCacheEntry> it = (Iterator)map.allEntries().iterator();
+        Iterator<GridCacheMapEntry> it = allEntries().iterator();
 
         GridCacheObsoleteEntryExtras extras = new GridCacheObsoleteEntryExtras(clearVer);
 
         while (it.hasNext()) {
-            GridDhtCacheEntry cached = null;
+            GridCacheMapEntry cached = null;
 
             cctx.shared().database().checkpointReadLock();
 
             try {
                 cached = it.next();
 
-                if (cached.clearInternal(clearVer, extras)) {
-                    map.removeEntry(cached);
+                if (cached instanceof GridDhtCacheEntry && ((GridDhtCacheEntry)cached).clearInternal(clearVer, extras)) {
+                    removeEntry(cached);
 
                     if (!cached.isInternal()) {
                         if (rec) {
@@ -961,9 +883,10 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
                     try {
                         CacheDataRow row = it0.next();
 
-                        GridDhtCacheEntry cached = (GridDhtCacheEntry)cctx.cache().entryEx(row.key());
+                        GridCacheMapEntry cached = putEntryIfObsoleteOrAbsent(cctx.affinity().affinityTopologyVersion(),
+                            row.key(), null, true, false);
 
-                        if (cached.clearInternal(clearVer, extras)) {
+                        if (cached instanceof GridDhtCacheEntry && ((GridDhtCacheEntry)cached).clearInternal(clearVer, extras)) {
                             if (rec) {
                                 cctx.events().addEvent(cached.partition(),
                                     cached.key(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index bf2ab29..38cbb8c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -1960,7 +1960,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                 if (part == null)
                     continue;
 
-                int size = part.size();
+                int size = part.dataStore().size();
 
                 if (size >= threshold)
                     X.println(">>>   Local partition [part=" + part.id() + ", size=" + size + ']');

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
index 872afee..7c1c38b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
@@ -134,24 +134,6 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
         return dht().preloader();
     }
 
-    /** {@inheritDoc} */
-    @Override public GridCacheMapEntry entryEx(KeyCacheObject key, boolean touch) {
-        GridNearCacheEntry entry = null;
-
-        while (true) {
-            try {
-                entry = (GridNearCacheEntry)super.entryEx(key, touch);
-
-                entry.initializeFromDht(ctx.affinity().affinityTopologyVersion());
-
-                return entry;
-            }
-            catch (GridCacheEntryRemovedException ignore) {
-                if (log.isDebugEnabled())
-                    log.debug("Got removed near entry while initializing from DHT entry (will retry): " + entry);
-            }
-        }
-    }
 
     /** {@inheritDoc} */
     @Override public GridCacheMapEntry entryEx(KeyCacheObject key, AffinityTopologyVersion topVer) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index c29e01e..b72a7fb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -973,7 +973,6 @@ public final class DataStructuresProcessor extends GridProcessorAdapter implemen
         ccfg.setBackups(cfg.getBackups());
         ccfg.setCacheMode(cfg.getCacheMode());
         ccfg.setAtomicityMode(cfg.getAtomicityMode());
-        ccfg.setOffHeapMaxMemory(cfg.getOffHeapMaxMemory());
         ccfg.setNodeFilter(cfg.getNodeFilter());
         ccfg.setWriteSynchronizationMode(FULL_SYNC);
         ccfg.setAtomicWriteOrderMode(PRIMARY);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index 1753cfd..d7395df 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -144,19 +144,13 @@ public class PlatformConfigurationUtils {
         ccfg.setCacheMode(CacheMode.fromOrdinal(in.readInt()));
         ccfg.setCopyOnRead(in.readBoolean());
         ccfg.setEagerTtl(in.readBoolean());
-        ccfg.setEvictSynchronized(in.readBoolean());
-        ccfg.setEvictSynchronizedConcurrencyLevel(in.readInt());
-        ccfg.setEvictSynchronizedKeyBufferSize(in.readInt());
-        ccfg.setEvictSynchronizedTimeout(in.readLong());
         ccfg.setInvalidate(in.readBoolean());
         ccfg.setStoreKeepBinary(in.readBoolean());
         ccfg.setLoadPreviousValue(in.readBoolean());
         ccfg.setDefaultLockTimeout(in.readLong());
         ccfg.setLongQueryWarningTimeout(in.readLong());
         ccfg.setMaxConcurrentAsyncOperations(in.readInt());
-        ccfg.setEvictMaxOverflowRatio(in.readFloat());
         ccfg.setName(in.readString());
-        ccfg.setOffHeapMaxMemory(in.readLong());
         ccfg.setReadFromBackup(in.readBoolean());
         ccfg.setRebalanceBatchSize(in.readInt());
         ccfg.setRebalanceDelay(in.readLong());
@@ -164,7 +158,6 @@ public class PlatformConfigurationUtils {
         ccfg.setRebalanceThrottle(in.readLong());
         ccfg.setRebalanceTimeout(in.readLong());
         ccfg.setSqlEscapeAll(in.readBoolean());
-        ccfg.setSqlOnheapRowCacheSize(in.readInt());
         ccfg.setStartSize(in.readInt());
         ccfg.setWriteBehindBatchSize(in.readInt());
         ccfg.setWriteBehindEnabled(in.readBoolean());
@@ -197,6 +190,9 @@ public class PlatformConfigurationUtils {
             ccfg.setNearConfiguration(readNearConfiguration(in));
 
         ccfg.setEvictionPolicy(readEvictionPolicy(in));
+        if (ccfg.getEvictionPolicy() != null)
+            ccfg.setOnheapCacheEnabled(true);
+
         ccfg.setAffinity(readAffinityFunction(in));
         ccfg.setExpiryPolicyFactory(readExpiryPolicyFactory(in));
 
@@ -755,19 +751,13 @@ public class PlatformConfigurationUtils {
         writeEnumInt(writer, ccfg.getCacheMode(), CacheConfiguration.DFLT_CACHE_MODE);
         writer.writeBoolean(ccfg.isCopyOnRead());
         writer.writeBoolean(ccfg.isEagerTtl());
-        writer.writeBoolean(ccfg.isEvictSynchronized());
-        writer.writeInt(ccfg.getEvictSynchronizedConcurrencyLevel());
-        writer.writeInt(ccfg.getEvictSynchronizedKeyBufferSize());
-        writer.writeLong(ccfg.getEvictSynchronizedTimeout());
         writer.writeBoolean(ccfg.isInvalidate());
         writer.writeBoolean(ccfg.isStoreKeepBinary());
         writer.writeBoolean(ccfg.isLoadPreviousValue());
         writer.writeLong(ccfg.getDefaultLockTimeout());
         writer.writeLong(ccfg.getLongQueryWarningTimeout());
         writer.writeInt(ccfg.getMaxConcurrentAsyncOperations());
-        writer.writeFloat(ccfg.getEvictMaxOverflowRatio());
         writer.writeString(ccfg.getName());
-        writer.writeLong(ccfg.getOffHeapMaxMemory());
         writer.writeBoolean(ccfg.isReadFromBackup());
         writer.writeInt(ccfg.getRebalanceBatchSize());
         writer.writeLong(ccfg.getRebalanceDelay());
@@ -775,7 +765,6 @@ public class PlatformConfigurationUtils {
         writer.writeLong(ccfg.getRebalanceThrottle());
         writer.writeLong(ccfg.getRebalanceTimeout());
         writer.writeBoolean(ccfg.isSqlEscapeAll());
-        writer.writeInt(ccfg.getSqlOnheapRowCacheSize());
         writer.writeInt(ccfg.getStartSize());
         writer.writeInt(ccfg.getWriteBehindBatchSize());
         writer.writeBoolean(ccfg.isWriteBehindEnabled());

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
index 0f5cd46..e087881 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
@@ -127,7 +127,6 @@ public class VisorCacheConfiguration implements Serializable, LessNamingBean {
         writeSynchronizationMode = ccfg.getWriteSynchronizationMode();
         invalidate = ccfg.isInvalidate();
         startSize = ccfg.getStartSize();
-        offHeapMaxMemory = ccfg.getOffHeapMaxMemory();
         maxConcurrentAsyncOps = ccfg.getMaxConcurrentAsyncOperations();
         interceptor = compactClass(ccfg.getInterceptor());
         typeMeta = VisorCacheTypeMetadata.list(ccfg.getQueryEntities(), ccfg.getCacheStoreFactory());

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheEvictionConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheEvictionConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheEvictionConfiguration.java
index 05d5c38..b6f72c4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheEvictionConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheEvictionConfiguration.java
@@ -43,21 +43,6 @@ public class VisorCacheEvictionConfiguration implements Serializable, LessNaming
     /** Eviction filter to specify which entries should not be evicted. */
     private String filter;
 
-    /** Synchronous eviction concurrency level. */
-    private int syncConcurrencyLvl;
-
-    /** Synchronous eviction timeout. */
-    private long syncTimeout;
-
-    /** Synchronized key buffer size. */
-    private int syncKeyBufSize;
-
-    /** Synchronous evicts flag. */
-    private boolean evictSynchronized;
-
-    /** Eviction max overflow ratio. */
-    private float maxOverflowRatio;
-
     /**
      * @param ccfg Cache configuration.
      * @return Data transfer object for eviction configuration properties.
@@ -70,11 +55,6 @@ public class VisorCacheEvictionConfiguration implements Serializable, LessNaming
         cfg.plc = compactClass(plc);
         cfg.plcMaxSize = evictionPolicyMaxSize(plc);
         cfg.filter = compactClass(ccfg.getEvictionFilter());
-        cfg.syncConcurrencyLvl = ccfg.getEvictSynchronizedConcurrencyLevel();
-        cfg.syncTimeout = ccfg.getEvictSynchronizedTimeout();
-        cfg.syncKeyBufSize = ccfg.getEvictSynchronizedKeyBufferSize();
-        cfg.evictSynchronized = ccfg.isEvictSynchronized();
-        cfg.maxOverflowRatio = ccfg.getEvictMaxOverflowRatio();
 
         return cfg;
     }
@@ -100,41 +80,6 @@ public class VisorCacheEvictionConfiguration implements Serializable, LessNaming
         return filter;
     }
 
-    /**
-     * @return synchronized eviction concurrency level.
-     */
-    public int synchronizedConcurrencyLevel() {
-        return syncConcurrencyLvl;
-    }
-
-    /**
-     * @return synchronized eviction timeout.
-     */
-    public long synchronizedTimeout() {
-        return syncTimeout;
-    }
-
-    /**
-     * @return Synchronized key buffer size.
-     */
-    public int synchronizedKeyBufferSize() {
-        return syncKeyBufSize;
-    }
-
-    /**
-     * @return Synchronous evicts flag.
-     */
-    public boolean evictSynchronized() {
-        return evictSynchronized;
-    }
-
-    /**
-     * @return Eviction max overflow ratio.
-     */
-    public float maxOverflowRatio() {
-        return maxOverflowRatio;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorCacheEvictionConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryConfiguration.java
index d753b76..a21cf43 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryConfiguration.java
@@ -73,7 +73,6 @@ public class VisorCacheQueryConfiguration implements Serializable, LessNamingBea
         longQryWarnTimeout = ccfg.getLongQueryWarningTimeout();
         sqlEscapeAll = ccfg.isSqlEscapeAll();
         indexedTypes = compactClasses(ccfg.getIndexedTypes());
-        sqlOnheapRowCacheSize = ccfg.getSqlOnheapRowCacheSize();
         sqlSchema = ccfg.getSqlSchema();
 
         return this;

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsSelfTest.java
index 80c0887..2cb36f9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsSelfTest.java
@@ -23,7 +23,6 @@ import java.util.concurrent.CountDownLatch;
 import org.apache.ignite.GridTestTask;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy;
 import org.apache.ignite.cluster.ClusterMetrics;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -62,11 +61,8 @@ public class ClusterNodeMetricsSelfTest extends GridCommonAbstractTest {
     /** Amount of cache entries. */
     private static final int MAX_VALS_AMOUNT = 400;
 
-    /** With OFFHEAP_VALUES policy. */
-    private final String OFF_HEAP_VALUE_NAME = "offHeapValuesCfg";
-
-    /** With ONHEAP_TIERED policy. */
-    private final String ON_HEAP_TIERED_NAME = "onHeapTieredCfg";
+    /** Cache name. */
+    private final String CACHE_NAME = "cache1";
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
@@ -91,25 +87,18 @@ public class ClusterNodeMetricsSelfTest extends GridCommonAbstractTest {
         cfg.setCacheConfiguration();
         cfg.setMetricsUpdateFrequency(0);
 
-        CacheConfiguration<Integer, Object> offHeapValuesCfg = defaultCacheConfiguration();
-        offHeapValuesCfg.setName(OFF_HEAP_VALUE_NAME);
-        offHeapValuesCfg.setStatisticsEnabled(true);
-        offHeapValuesCfg.setMemoryMode(CacheMemoryMode.OFFHEAP_VALUES);
-        offHeapValuesCfg.setOffHeapMaxMemory(MAX_VALS_AMOUNT * VAL_SIZE);
-
-        CacheConfiguration<Integer, Object> onHeapTieredCfg = defaultCacheConfiguration();
-        onHeapTieredCfg.setName(ON_HEAP_TIERED_NAME);
-        onHeapTieredCfg.setStatisticsEnabled(true);
-        onHeapTieredCfg.setMemoryMode(CacheMemoryMode.ONHEAP_TIERED);
-        onHeapTieredCfg.setOffHeapMaxMemory(MAX_VALS_AMOUNT * VAL_SIZE);
+        CacheConfiguration<Integer, Object> ccfg = defaultCacheConfiguration();
+        ccfg.setName(CACHE_NAME);
+        ccfg.setStatisticsEnabled(true);
 
         FifoEvictionPolicy plc = new FifoEvictionPolicy();
         plc.setMaxMemorySize(MAX_VALS_AMOUNT * VAL_SIZE);
         plc.setMaxSize(0);
 
-        onHeapTieredCfg.setEvictionPolicy(plc);
+        ccfg.setEvictionPolicy(plc);
+        ccfg.setOnheapCacheEnabled(true);
 
-        return cfg.setCacheConfiguration(offHeapValuesCfg, onHeapTieredCfg);
+        return cfg.setCacheConfiguration(ccfg);
     }
 
     /**
@@ -120,32 +109,25 @@ public class ClusterNodeMetricsSelfTest extends GridCommonAbstractTest {
 
         Ignite ignite = grid();
 
-        final IgniteCache onHeapCache = ignite.getOrCreateCache(ON_HEAP_TIERED_NAME);
-        final IgniteCache offHeapCache = ignite.getOrCreateCache(OFF_HEAP_VALUE_NAME);
+        final IgniteCache cache = ignite.getOrCreateCache(CACHE_NAME);
 
-        long prevTieredOffHeapSize = onHeapCache.metrics().getOffHeapAllocatedSize();
-        long prevValuesOffHeapSize = offHeapCache.metrics().getOffHeapAllocatedSize();
+        long prevTieredOffHeapSize = cache.metrics().getOffHeapAllocatedSize();
 
         assertEquals(0, prevTieredOffHeapSize);
-        assertEquals(0, prevValuesOffHeapSize);
 
         long prevClusterNonHeapMemoryUsed = ignite.cluster().metrics().getNonHeapMemoryUsed();
 
-        fillCache(onHeapCache);
+        fillCache(cache);
 
-        assertTrue(onHeapCache.metrics().getOffHeapAllocatedSize() > (MAX_VALS_AMOUNT - 5)
+        assertTrue(cache.metrics().getOffHeapAllocatedSize() > (MAX_VALS_AMOUNT - 5)
             * VAL_SIZE + prevTieredOffHeapSize);
-        assertEquals(0, offHeapCache.metrics().getOffHeapAllocatedSize());
 
         assertTrue(prevClusterNonHeapMemoryUsed < ignite.cluster().metrics().getNonHeapMemoryUsed());
 
         prevClusterNonHeapMemoryUsed = ignite.cluster().metrics().getNonHeapMemoryUsed();
-        prevTieredOffHeapSize = onHeapCache.metrics().getOffHeapAllocatedSize();
-
-        fillCache(offHeapCache);
+        prevTieredOffHeapSize = cache.metrics().getOffHeapAllocatedSize();
 
-        assertTrue(offHeapCache.metrics().getOffHeapAllocatedSize() > (MAX_VALS_AMOUNT - 5) * VAL_SIZE);
-        assertEquals(prevTieredOffHeapSize, onHeapCache.metrics().getOffHeapAllocatedSize());
+        assertEquals(prevTieredOffHeapSize, cache.metrics().getOffHeapAllocatedSize());
         assertTrue((MAX_VALS_AMOUNT - 5) * VAL_SIZE + prevClusterNonHeapMemoryUsed <
             ignite.cluster().metrics().getNonHeapMemoryUsed());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/BinaryObjectOffHeapUnswapTemporaryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/BinaryObjectOffHeapUnswapTemporaryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/BinaryObjectOffHeapUnswapTemporaryTest.java
deleted file mode 100644
index 99133b8..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/BinaryObjectOffHeapUnswapTemporaryTest.java
+++ /dev/null
@@ -1,365 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache;
-
-import java.util.Collection;
-import java.util.LinkedHashSet;
-import java.util.Map;
-import java.util.Set;
-import javax.cache.processor.EntryProcessorException;
-import javax.cache.processor.MutableEntry;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.binary.BinaryObject;
-import org.apache.ignite.binary.BinaryObjectBuilder;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheEntry;
-import org.apache.ignite.cache.CacheEntryProcessor;
-import org.apache.ignite.cache.CacheMemoryMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.binary.BinaryMarshaller;
-import org.apache.ignite.internal.binary.BinaryObjectOffheapImpl;
-import org.apache.ignite.lang.IgniteInClosure;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.apache.ignite.transactions.Transaction;
-import org.apache.ignite.transactions.TransactionConcurrency;
-import org.apache.ignite.transactions.TransactionIsolation;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_VALUES;
-
-/**
- *
- */
-public class BinaryObjectOffHeapUnswapTemporaryTest extends GridCommonAbstractTest {
-    /** */
-    private static final int CNT = 20;
-
-    /** Cache name. */
-    private static final String CACHE_NAME = "cache";
-
-    /** */
-    @SuppressWarnings("serial")
-    private static final CacheEntryProcessor PROC = new CacheEntryProcessor() {
-        @Override public Object process(MutableEntry entry, Object... arguments) throws EntryProcessorException {
-            return entry.getValue();
-        }
-    };
-
-    /** */
-    private CacheAtomicityMode atomicityMode;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
-
-        c.setMarshaller(new BinaryMarshaller());
-
-        return c;
-    }
-
-    /**
-     * @param atomicityMode Atomicity mode.
-     * @param memoryMode Memory mode.
-     * @return Cache configuration.
-     */
-    private CacheConfiguration<Object, Object> cacheConfiguration(CacheAtomicityMode atomicityMode,
-        CacheMemoryMode memoryMode) {
-        this.atomicityMode = atomicityMode;
-
-        CacheConfiguration<Object, Object>  cfg = new CacheConfiguration<>();
-
-        cfg.setName(CACHE_NAME);
-        cfg.setCacheMode(CacheMode.PARTITIONED);
-        cfg.setAtomicityMode(atomicityMode);
-        cfg.setMemoryMode(memoryMode);
-        cfg.setBackups(1);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-
-        startGridsMultiThreaded(3);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        super.afterTestsStopped();
-
-        stopAllGrids();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTxOffheapTiered() throws Exception {
-        ignite(0).getOrCreateCache(cacheConfiguration(TRANSACTIONAL, OFFHEAP_TIERED));
-
-        try {
-            doTest();
-        }
-        finally {
-            ignite(0).destroyCache(null);
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTxOffheapValues() throws Exception {
-        ignite(0).getOrCreateCache(cacheConfiguration(TRANSACTIONAL, OFFHEAP_VALUES));
-
-        try {
-            doTest();
-        }
-        finally {
-            ignite(0).destroyCache(null);
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAtomicOffheapTiered() throws Exception {
-        ignite(0).getOrCreateCache(cacheConfiguration(ATOMIC, OFFHEAP_TIERED));
-
-        try {
-            doTest();
-        }
-        finally {
-            ignite(0).destroyCache(null);
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAtomicOffheapValues() throws Exception {
-        ignite(0).getOrCreateCache(cacheConfiguration(ATOMIC, OFFHEAP_VALUES));
-
-        try {
-            doTest();
-        }
-        finally {
-            ignite(0).destroyCache(null);
-        }
-    }
-
-    /**
-     *
-     */
-    @SuppressWarnings("serial")
-    private void doTest() {
-        final IgniteCache<Integer, BinaryObject> cache = jcache(0, CACHE_NAME).withKeepBinary();
-
-        for (int key = 0; key < CNT; key++)
-            jcache(0, CACHE_NAME).put(key, new TestObject(key));
-
-        for (int key = CNT; key < 2 * CNT; key++) {
-            BinaryObjectBuilder builder = ignite(0).binary().builder("SomeType");
-            builder.setField("field1", key);
-            builder.setField("field2", "name_" + key);
-
-            cache.put(key, builder.build());
-        }
-
-        Set<Integer> keys = new LinkedHashSet<>();
-
-        for (int i = 0; i < 2 * CNT; i++)
-            keys.add(i);
-
-        check(new IgniteInClosure<Integer>() {
-            @Override public void apply(Integer key) {
-                assertFalse(cache.get(key) instanceof BinaryObjectOffheapImpl);
-            }
-        });
-
-        check(new IgniteInClosure<Integer>() {
-            @Override public void apply(Integer key) {
-                assertFalse(cache.getEntry(key).getValue() instanceof BinaryObjectOffheapImpl);
-            }
-        });
-
-        check(new IgniteInClosure<Integer>() {
-            @Override public void apply(Integer key) {
-                assertFalse(cache.getAndPut(key, cache.get(key)) instanceof BinaryObjectOffheapImpl);
-            }
-        });
-
-        check(new IgniteInClosure<Integer>() {
-            @Override public void apply(Integer key) {
-                assertFalse(cache.getAndReplace(key, cache.get(key)) instanceof BinaryObjectOffheapImpl);
-            }
-        });
-
-        check(new IgniteInClosure<Integer>() {
-            @Override public void apply(Integer key) {
-                assertFalse(cache.getAndPutIfAbsent(key, cache.get(key)) instanceof BinaryObjectOffheapImpl);
-            }
-        });
-
-        check(new IgniteInClosure<Integer>() {
-            @Override public void apply(Integer key) {
-                assertFalse(cache.localPeek(key) instanceof BinaryObjectOffheapImpl);
-            }
-        });
-
-        check(new IgniteInClosure<Integer>() {
-            @Override public void apply(Integer key) {
-                assertFalse(cache.getAndRemove(key) instanceof BinaryObjectOffheapImpl);
-            }
-        });
-
-        check(new IgniteInClosure<Integer>() {
-            @Override public void apply(Integer key) {
-                assertFalse(cache.invoke(key, PROC) instanceof BinaryObjectOffheapImpl);
-            }
-        });
-
-        // GetAll.
-        Map<Integer, BinaryObject> res = cache.getAll(keys);
-
-        for (BinaryObject val : res.values())
-            assertFalse(val instanceof BinaryObjectOffheapImpl);
-
-        if (atomicityMode == TRANSACTIONAL) {
-            for (TransactionIsolation isolation : TransactionIsolation.values()) {
-                for (TransactionConcurrency concurrency : TransactionConcurrency.values()) {
-                    try (Transaction tx = ignite(0).transactions().txStart(concurrency, isolation)) {
-                        res = cache.getAll(keys);
-
-                        for (BinaryObject val : res.values())
-                            assertFalse(val instanceof BinaryObjectOffheapImpl);
-
-                        tx.commit();
-                    }
-                }
-            }
-        }
-
-        // GetAllOutTx.
-        res = cache.getAllOutTx(keys);
-
-        for (BinaryObject val : res.values())
-            assertFalse(val instanceof BinaryObjectOffheapImpl);
-
-        if (atomicityMode == TRANSACTIONAL) {
-            for (TransactionIsolation isolation : TransactionIsolation.values()) {
-                for (TransactionConcurrency concurrency : TransactionConcurrency.values()) {
-                    try (Transaction tx = ignite(0).transactions().txStart(concurrency, isolation)) {
-                        res = cache.getAllOutTx(keys);
-
-                        for (BinaryObject val : res.values())
-                            assertFalse(val instanceof BinaryObjectOffheapImpl);
-
-                        tx.commit();
-                    }
-                }
-            }
-        }
-
-        // InvokeAll.
-        res = cache.invokeAll(keys, PROC);
-
-        for (BinaryObject val : res.values())
-            assertFalse(val instanceof BinaryObjectOffheapImpl);
-
-        if (atomicityMode == TRANSACTIONAL) {
-            for (TransactionIsolation isolation : TransactionIsolation.values()) {
-                for (TransactionConcurrency concurrency : TransactionConcurrency.values()) {
-                    try (Transaction tx = ignite(0).transactions().txStart(concurrency, isolation)) {
-                        res = cache.invokeAll(keys, PROC);
-
-                        for (BinaryObject val : res.values())
-                            assertFalse(val instanceof BinaryObjectOffheapImpl);
-
-                        tx.commit();
-                    }
-                }
-            }
-        }
-
-        // GetEntries.
-        Collection<CacheEntry<Integer, BinaryObject>> entries = cache.getEntries(keys);
-
-        for (CacheEntry<Integer, BinaryObject> e : entries)
-            assertFalse(e.getValue() instanceof BinaryObjectOffheapImpl);
-
-        if (atomicityMode == TRANSACTIONAL) {
-            for (TransactionIsolation isolation : TransactionIsolation.values()) {
-                for (TransactionConcurrency concurrency : TransactionConcurrency.values()) {
-                    try (Transaction tx = ignite(0).transactions().txStart(concurrency, isolation)) {
-                        entries = cache.getEntries(keys);
-
-                        for (CacheEntry<Integer, BinaryObject> e : entries)
-                            assertFalse(e.getValue() instanceof BinaryObjectOffheapImpl);
-
-                        tx.commit();
-                    }
-                }
-            }
-        }
-    }
-
-    /**
-     *
-     */
-    private void check(IgniteInClosure<Integer> checkOp) {
-        for (int key = 0; key < 2 * CNT; key++) {
-            checkOp.apply(key);
-
-            if (atomicityMode == TRANSACTIONAL) {
-                for (TransactionIsolation isolation : TransactionIsolation.values()) {
-                    for (TransactionConcurrency concurrency : TransactionConcurrency.values()) {
-                        try (Transaction tx = ignite(0).transactions().txStart(concurrency, isolation)) {
-                            checkOp.apply(key);
-
-                            tx.commit();
-                        }
-                    }
-                }
-            }
-        }
-    }
-
-    /**
-     *
-     */
-    @SuppressWarnings("PackageVisibleField")
-    private static class TestObject {
-        /** */
-        String field;
-
-        /** */
-        int field2;
-
-        /**
-         * @param key Key.
-         */
-        TestObject(int key) {
-            field = "str" + key;
-            field2 = key;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationLeakTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationLeakTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationLeakTest.java
index 5c85d01..df9fbd5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationLeakTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationLeakTest.java
@@ -70,6 +70,7 @@ public class CacheConfigurationLeakTest extends GridCommonAbstractTest {
                     CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
                     ccfg.setName("cache-" + idx + "-" + i);
                     ccfg.setEvictionPolicy(new LruEvictionPolicy(1000));
+                    ccfg.setOnheapCacheEnabled(true);
 
                     IgniteCache<Object, Object> cache = ignite.createCache(ccfg);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDeferredDeleteQueueTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDeferredDeleteQueueTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDeferredDeleteQueueTest.java
index b764d5b..37d3f4d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDeferredDeleteQueueTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDeferredDeleteQueueTest.java
@@ -116,7 +116,7 @@ public class CacheDeferredDeleteQueueTest extends GridCommonAbstractTest {
                         for (GridDhtLocalPartition p : top.currentLocalPartitions()) {
                             Collection<Object> rmvQueue = GridTestUtils.getFieldValue(p, "rmvQueue");
 
-                            if (!rmvQueue.isEmpty() || p.size() != 0)
+                            if (!rmvQueue.isEmpty() || p.dataStore().size() != 0)
                                 return false;
                         }
                     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDhtLocalPartitionAfterRemoveSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDhtLocalPartitionAfterRemoveSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDhtLocalPartitionAfterRemoveSelfTest.java
index 32c31d3..1da94d4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDhtLocalPartitionAfterRemoveSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDhtLocalPartitionAfterRemoveSelfTest.java
@@ -76,7 +76,7 @@ public class CacheDhtLocalPartitionAfterRemoveSelfTest extends GridCommonAbstrac
             cache = grid(g).cache(null);
 
             for (GridDhtLocalPartition p : dht(cache).topology().localPartitions()) {
-                int size = p.size();
+                int size = p.dataStore().size();
 
                 assertTrue("Unexpected size: " + size, size <= 32);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
index 5829724..2017365 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
@@ -22,7 +22,6 @@ import javax.cache.processor.MutableEntry;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -37,9 +36,6 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.PRIMARY;
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_VALUES;
-import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 
@@ -95,25 +91,7 @@ public abstract class CacheEnumOperationsAbstractTest extends GridCommonAbstract
      * @throws Exception If failed.
      */
     public void testAtomic() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC, ONHEAP_TIERED);
-
-        enumOperations(ccfg);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAtomicOffheapValues() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC, OFFHEAP_VALUES);
-
-        enumOperations(ccfg);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAtomicOffheapTiered() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC, OFFHEAP_TIERED);
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC);
 
         enumOperations(ccfg);
     }
@@ -122,25 +100,7 @@ public abstract class CacheEnumOperationsAbstractTest extends GridCommonAbstract
      * @throws Exception If failed.
      */
     public void testTx() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC, ONHEAP_TIERED);
-
-        enumOperations(ccfg);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTxOffheapValues() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC, OFFHEAP_VALUES);
-
-        enumOperations(ccfg);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTxOffheapTiered() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC, OFFHEAP_TIERED);
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC);
 
         enumOperations(ccfg);
     }
@@ -269,25 +229,19 @@ public abstract class CacheEnumOperationsAbstractTest extends GridCommonAbstract
      * @param cacheMode Cache mode.
      * @param backups Number of backups.
      * @param atomicityMode Cache atomicity mode.
-     * @param memoryMode Cache memory mode.
      * @return Cache configuration.
      */
     private CacheConfiguration<Object, Object> cacheConfiguration(
         CacheMode cacheMode,
         int backups,
-        CacheAtomicityMode atomicityMode,
-        CacheMemoryMode memoryMode) {
+        CacheAtomicityMode atomicityMode) {
         CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
 
         ccfg.setAtomicityMode(atomicityMode);
         ccfg.setCacheMode(cacheMode);
-        ccfg.setMemoryMode(memoryMode);
         ccfg.setWriteSynchronizationMode(FULL_SYNC);
         ccfg.setAtomicWriteOrderMode(PRIMARY);
 
-        if (memoryMode == OFFHEAP_TIERED)
-            ccfg.setOffHeapMaxMemory(0);
-
         if (cacheMode == PARTITIONED)
             ccfg.setBackups(backups);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryAbstractTest.java
index daec5a9..0ddac75 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryAbstractTest.java
@@ -171,6 +171,8 @@ public abstract class CacheGetEntryAbstractTest extends GridCacheAbstractSelfTes
      * @throws Exception If failed.
      */
     public void testLocalTransactional() throws Exception {
+        // TODO: fails since d13520e9a05bd9e9b987529472d6317951b72f96, need to review changes.
+
         CacheConfiguration cfg = new CacheConfiguration();
 
         cfg.setWriteSynchronizationMode(FULL_SYNC);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheInterceptorPartitionCounterLocalSanityTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheInterceptorPartitionCounterLocalSanityTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheInterceptorPartitionCounterLocalSanityTest.java
index 5ca0e29..5fe7d70 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheInterceptorPartitionCounterLocalSanityTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheInterceptorPartitionCounterLocalSanityTest.java
@@ -33,7 +33,6 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheInterceptorAdapter;
 import org.apache.ignite.cache.CacheInterceptorEntry;
-import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.store.CacheStore;
 import org.apache.ignite.cache.store.CacheStoreAdapter;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -53,7 +52,6 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
@@ -114,10 +112,7 @@ public class CacheInterceptorPartitionCounterLocalSanityTest extends GridCommonA
      * @throws Exception If failed.
      */
     public void testLocal() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(
-            ATOMIC,
-            ONHEAP_TIERED,
-            false);
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(ATOMIC,false);
 
         doTestPartitionCounterOperation(ccfg);
     }
@@ -126,10 +121,7 @@ public class CacheInterceptorPartitionCounterLocalSanityTest extends GridCommonA
      * @throws Exception If failed.
      */
     public void testLocalWithStore() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(
-            ATOMIC,
-            ONHEAP_TIERED,
-            true);
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(ATOMIC,true);
 
         doTestPartitionCounterOperation(ccfg);
     }
@@ -138,10 +130,7 @@ public class CacheInterceptorPartitionCounterLocalSanityTest extends GridCommonA
      * @throws Exception If failed.
      */
     public void testLocalTx() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(
-            TRANSACTIONAL,
-            ONHEAP_TIERED,
-            false);
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(TRANSACTIONAL,false);
 
         doTestPartitionCounterOperation(ccfg);
     }
@@ -150,10 +139,7 @@ public class CacheInterceptorPartitionCounterLocalSanityTest extends GridCommonA
      * @throws Exception If failed.
      */
     public void testLocalTxWithStore() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(
-            TRANSACTIONAL,
-            ONHEAP_TIERED,
-            true);
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(TRANSACTIONAL,true);
 
         doTestPartitionCounterOperation(ccfg);
     }
@@ -488,18 +474,15 @@ public class CacheInterceptorPartitionCounterLocalSanityTest extends GridCommonA
 
     /**
      * @param atomicityMode Cache atomicity mode.
-     * @param memoryMode Cache memory mode.
      * @param store If {@code true} configures dummy cache store.
      * @return Cache configuration.
      */
     protected CacheConfiguration<Object, Object> cacheConfiguration(
         CacheAtomicityMode atomicityMode,
-        CacheMemoryMode memoryMode,
         boolean store) {
         CacheConfiguration<TestKey, TestValue> ccfg = new CacheConfiguration<>();
 
         ccfg.setAtomicityMode(atomicityMode);
-        ccfg.setMemoryMode(memoryMode);
         ccfg.setCacheMode(LOCAL);
 
         if (store) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheInterceptorPartitionCounterRandomOperationsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheInterceptorPartitionCounterRandomOperationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheInterceptorPartitionCounterRandomOperationsTest.java
index 97a815b..5b7769e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheInterceptorPartitionCounterRandomOperationsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheInterceptorPartitionCounterRandomOperationsTest.java
@@ -39,7 +39,6 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheInterceptorAdapter;
 import org.apache.ignite.cache.CacheInterceptorEntry;
-import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cache.store.CacheStore;
@@ -64,9 +63,6 @@ import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.PRIMARY;
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_VALUES;
-import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
@@ -153,7 +149,6 @@ public class CacheInterceptorPartitionCounterRandomOperationsTest extends GridCo
         CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
             1,
             ATOMIC,
-            ONHEAP_TIERED,
             false);
 
         doTestPartitionCounterOperation(ccfg);
@@ -166,7 +161,6 @@ public class CacheInterceptorPartitionCounterRandomOperationsTest extends GridCo
         CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
             1,
             ATOMIC,
-            ONHEAP_TIERED,
             true);
 
         doTestPartitionCounterOperation(ccfg);
@@ -179,7 +173,6 @@ public class CacheInterceptorPartitionCounterRandomOperationsTest extends GridCo
         CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED,
             0,
             ATOMIC,
-            ONHEAP_TIERED,
             false);
 
         doTestPartitionCounterOperation(ccfg);
@@ -192,7 +185,6 @@ public class CacheInterceptorPartitionCounterRandomOperationsTest extends GridCo
         CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED,
             0,
             ATOMIC,
-            ONHEAP_TIERED,
             true);
 
         doTestPartitionCounterOperation(ccfg);
@@ -201,50 +193,10 @@ public class CacheInterceptorPartitionCounterRandomOperationsTest extends GridCo
     /**
      * @throws Exception If failed.
      */
-    public void testAtomicOffheapValues() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
-            1,
-            ATOMIC,
-            OFFHEAP_VALUES,
-            false);
-
-        doTestPartitionCounterOperation(ccfg);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAtomicOffheapValuesWithStore() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
-            1,
-            ATOMIC,
-            OFFHEAP_VALUES,
-            true);
-
-        doTestPartitionCounterOperation(ccfg);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAtomicOffheapTiered() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
-            1,
-            ATOMIC,
-            OFFHEAP_TIERED,
-            false);
-
-        doTestPartitionCounterOperation(ccfg);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     public void testAtomicNoBackups() throws Exception {
         CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
             0,
             ATOMIC,
-            ONHEAP_TIERED,
             false);
 
         doTestPartitionCounterOperation(ccfg);
@@ -257,7 +209,6 @@ public class CacheInterceptorPartitionCounterRandomOperationsTest extends GridCo
         CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
             1,
             TRANSACTIONAL,
-            ONHEAP_TIERED,
             false);
 
         doTestPartitionCounterOperation(ccfg);
@@ -270,7 +221,6 @@ public class CacheInterceptorPartitionCounterRandomOperationsTest extends GridCo
         CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
             1,
             TRANSACTIONAL,
-            ONHEAP_TIERED,
             true);
 
         doTestPartitionCounterOperation(ccfg);
@@ -283,7 +233,6 @@ public class CacheInterceptorPartitionCounterRandomOperationsTest extends GridCo
         CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
             1,
             TRANSACTIONAL,
-            ONHEAP_TIERED,
             false);
 
         doTestPartitionCounterOperation(ccfg);
@@ -296,7 +245,6 @@ public class CacheInterceptorPartitionCounterRandomOperationsTest extends GridCo
         CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED,
             0,
             TRANSACTIONAL,
-            ONHEAP_TIERED,
             false);
 
         doTestPartitionCounterOperation(ccfg);
@@ -309,7 +257,6 @@ public class CacheInterceptorPartitionCounterRandomOperationsTest extends GridCo
         CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED,
             0,
             TRANSACTIONAL,
-            ONHEAP_TIERED,
             true);
 
         doTestPartitionCounterOperation(ccfg);
@@ -318,63 +265,10 @@ public class CacheInterceptorPartitionCounterRandomOperationsTest extends GridCo
     /**
      * @throws Exception If failed.
      */
-    public void testTxOffheapValues() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
-            1,
-            TRANSACTIONAL,
-            OFFHEAP_VALUES,
-            false);
-
-        doTestPartitionCounterOperation(ccfg);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTxOffheapValuesExplicit() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
-            1,
-            TRANSACTIONAL,
-            OFFHEAP_VALUES,
-            false);
-
-        doTestPartitionCounterOperation(ccfg);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTxOffheapTiered() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
-            1,
-            TRANSACTIONAL,
-            OFFHEAP_TIERED,
-            false);
-
-        doTestPartitionCounterOperation(ccfg);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTxOffheapTieredExplicit() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
-            1,
-            TRANSACTIONAL,
-            OFFHEAP_TIERED,
-            false);
-
-        doTestPartitionCounterOperation(ccfg);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     public void testTxNoBackups() throws Exception {
         CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
             0,
             TRANSACTIONAL,
-            ONHEAP_TIERED,
             false);
 
         doTestPartitionCounterOperation(ccfg);
@@ -387,7 +281,6 @@ public class CacheInterceptorPartitionCounterRandomOperationsTest extends GridCo
         CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
             0,
             TRANSACTIONAL,
-            ONHEAP_TIERED,
             true);
 
         doTestPartitionCounterOperation(ccfg);
@@ -400,7 +293,6 @@ public class CacheInterceptorPartitionCounterRandomOperationsTest extends GridCo
         CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
             0,
             TRANSACTIONAL,
-            ONHEAP_TIERED,
             false);
 
         doTestPartitionCounterOperation(ccfg);
@@ -824,7 +716,6 @@ public class CacheInterceptorPartitionCounterRandomOperationsTest extends GridCo
      * @param cacheMode Cache mode.
      * @param backups Number of backups.
      * @param atomicityMode Cache atomicity mode.
-     * @param memoryMode Cache memory mode.
      * @param store If {@code true} configures dummy cache store.
      * @return Cache configuration.
      */
@@ -832,13 +723,11 @@ public class CacheInterceptorPartitionCounterRandomOperationsTest extends GridCo
         CacheMode cacheMode,
         int backups,
         CacheAtomicityMode atomicityMode,
-        CacheMemoryMode memoryMode,
         boolean store) {
         CacheConfiguration<TestKey, TestValue> ccfg = new CacheConfiguration<>();
 
         ccfg.setAtomicityMode(atomicityMode);
         ccfg.setCacheMode(cacheMode);
-        ccfg.setMemoryMode(memoryMode);
         ccfg.setWriteSynchronizationMode(FULL_SYNC);
         ccfg.setAtomicWriteOrderMode(PRIMARY);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheNearReaderUpdateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheNearReaderUpdateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheNearReaderUpdateTest.java
index 4ea74ca..9bd7a8c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheNearReaderUpdateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheNearReaderUpdateTest.java
@@ -145,17 +145,6 @@ public class CacheNearReaderUpdateTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void testOneBackupOffheap() throws Exception {
-        CacheConfiguration<Integer, Integer> ccfg = cacheConfiguration(PARTITIONED, FULL_SYNC, 1, false, false);
-
-        GridTestUtils.setMemoryMode(null, ccfg, GridTestUtils.TestMemoryMode.OFFHEAP_TIERED, 0, 0);
-
-        testGetUpdateMultithreaded(ccfg);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     public void testGetUpdateMultithreaded(CacheConfiguration<Integer, Integer> ccfg) throws Exception {
         final List<Ignite> putNodes = new ArrayList<>();
 
@@ -327,7 +316,6 @@ public class CacheNearReaderUpdateTest extends GridCommonAbstractTest {
             ", near=" + (ccfg.getNearConfiguration() != null) +
             ", store=" + ccfg.isWriteThrough() +
             ", evictPlc=" + (ccfg.getEvictionPolicy() != null) +
-            ", maxOffheap=" + ccfg.getOffHeapMaxMemory()  +
             ']');
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheOffheapMapEntrySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheOffheapMapEntrySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheOffheapMapEntrySelfTest.java
index 1cb52c3..06fafb3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheOffheapMapEntrySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheOffheapMapEntrySelfTest.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.IgniteKernal;
@@ -30,9 +29,6 @@ import org.apache.ignite.internal.processors.cache.local.GridLocalCacheEntry;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_VALUES;
-import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
@@ -67,26 +63,23 @@ public class CacheOffheapMapEntrySelfTest extends GridCacheAbstractSelfTest {
     }
 
     /**
-     * @param igniteInstanceName Ignite instance name.
-     * @param memoryMode Memory mode.
+     * @param gridName Grid name.
      * @param atomicityMode Atomicity mode.
      * @param cacheMode Cache mode.
      * @param cacheName Cache name.
      * @return Cache configuration.
      * @throws Exception If failed.
      */
-    private CacheConfiguration cacheConfiguration(String igniteInstanceName,
-        CacheMemoryMode memoryMode,
+    private CacheConfiguration cacheConfiguration(String gridName,
         CacheAtomicityMode atomicityMode,
         CacheMode cacheMode,
         String cacheName)
         throws Exception
     {
-        CacheConfiguration cfg = super.cacheConfiguration(igniteInstanceName);
+        CacheConfiguration cfg = super.cacheConfiguration(gridName);
 
         cfg.setCacheMode(cacheMode);
         cfg.setAtomicityMode(atomicityMode);
-        cfg.setMemoryMode(memoryMode);
         cfg.setName(cacheName);
 
         return cfg;
@@ -96,46 +89,33 @@ public class CacheOffheapMapEntrySelfTest extends GridCacheAbstractSelfTest {
      * @throws Exception If failed.
      */
     public void testCacheMapEntry() throws Exception {
-        checkCacheMapEntry(ONHEAP_TIERED, ATOMIC, LOCAL, GridLocalCacheEntry.class);
+        checkCacheMapEntry(ATOMIC, LOCAL, GridLocalCacheEntry.class);
 
-        checkCacheMapEntry(OFFHEAP_TIERED, ATOMIC, LOCAL, GridLocalCacheEntry.class);
+        checkCacheMapEntry(TRANSACTIONAL, LOCAL, GridLocalCacheEntry.class);
 
-        checkCacheMapEntry(OFFHEAP_VALUES, ATOMIC, LOCAL, GridLocalCacheEntry.class);
+        checkCacheMapEntry(ATOMIC, PARTITIONED, GridNearCacheEntry.class);
 
-        checkCacheMapEntry(ONHEAP_TIERED, TRANSACTIONAL, LOCAL, GridLocalCacheEntry.class);
+        checkCacheMapEntry(TRANSACTIONAL, PARTITIONED, GridNearCacheEntry.class);
 
-        checkCacheMapEntry(OFFHEAP_TIERED, TRANSACTIONAL, LOCAL, GridLocalCacheEntry.class);
+        checkCacheMapEntry(ATOMIC, REPLICATED, GridDhtAtomicCacheEntry.class);
 
-        checkCacheMapEntry(OFFHEAP_VALUES, TRANSACTIONAL, LOCAL, GridLocalCacheEntry.class);
-
-        checkCacheMapEntry(ONHEAP_TIERED, ATOMIC, PARTITIONED, GridNearCacheEntry.class);
-
-        checkCacheMapEntry(ONHEAP_TIERED, TRANSACTIONAL, PARTITIONED, GridNearCacheEntry.class);
-
-        checkCacheMapEntry(ONHEAP_TIERED, ATOMIC, REPLICATED, GridDhtAtomicCacheEntry.class);
-
-        checkCacheMapEntry(ONHEAP_TIERED, TRANSACTIONAL, REPLICATED, GridDhtColocatedCacheEntry.class);
+        checkCacheMapEntry(TRANSACTIONAL, REPLICATED, GridDhtColocatedCacheEntry.class);
     }
 
     /**
-     * @param memoryMode Cache memory mode.
      * @param atomicityMode Cache atomicity mode.
      * @param cacheMode Cache mode.
      * @param entryCls Class of cache map entry.
      * @throws Exception If failed.
      */
-    private void checkCacheMapEntry(CacheMemoryMode memoryMode,
-        CacheAtomicityMode atomicityMode,
+    private void checkCacheMapEntry(CacheAtomicityMode atomicityMode,
         CacheMode cacheMode,
         Class<?> entryCls)
         throws Exception
     {
-        log.info("Test cache [memMode=" + memoryMode +
-            ", atomicityMode=" + atomicityMode +
-            ", cacheMode=" + cacheMode + ']');
+        log.info("Test cache [atomicityMode=" + atomicityMode + ", cacheMode=" + cacheMode + ']');
 
         CacheConfiguration cfg = cacheConfiguration(grid(0).name(),
-            memoryMode,
             atomicityMode,
             cacheMode,
             "Cache");

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CachePutEventListenerErrorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CachePutEventListenerErrorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CachePutEventListenerErrorSelfTest.java
index bd43956..f609af3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CachePutEventListenerErrorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CachePutEventListenerErrorSelfTest.java
@@ -23,7 +23,6 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -96,65 +95,36 @@ public class CachePutEventListenerErrorSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPartitionedAtomicOnHeap() throws Exception {
-        doTest(CacheMode.PARTITIONED, CacheAtomicityMode.ATOMIC, CacheMemoryMode.ONHEAP_TIERED);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPartitionedAtomicOffHeap() throws Exception {
-        doTest(CacheMode.PARTITIONED, CacheAtomicityMode.ATOMIC, CacheMemoryMode.OFFHEAP_TIERED);
+        doTest(CacheMode.PARTITIONED, CacheAtomicityMode.ATOMIC);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testPartitionedTransactionalOnHeap() throws Exception {
-        doTest(CacheMode.PARTITIONED, CacheAtomicityMode.TRANSACTIONAL, CacheMemoryMode.ONHEAP_TIERED);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPartitionedTransactionalOffHeap() throws Exception {
-        doTest(CacheMode.PARTITIONED, CacheAtomicityMode.TRANSACTIONAL, CacheMemoryMode.OFFHEAP_TIERED);
+        doTest(CacheMode.PARTITIONED, CacheAtomicityMode.TRANSACTIONAL);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testReplicatedAtomicOnHeap() throws Exception {
-        doTest(CacheMode.REPLICATED, CacheAtomicityMode.ATOMIC, CacheMemoryMode.ONHEAP_TIERED);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testReplicatedAtomicOffHeap() throws Exception {
-        doTest(CacheMode.REPLICATED, CacheAtomicityMode.ATOMIC, CacheMemoryMode.OFFHEAP_TIERED);
+        doTest(CacheMode.REPLICATED, CacheAtomicityMode.ATOMIC);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testReplicatedTransactionalOnHeap() throws Exception {
-        doTest(CacheMode.REPLICATED, CacheAtomicityMode.TRANSACTIONAL, CacheMemoryMode.ONHEAP_TIERED);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testReplicatedTransactionalOffHeap() throws Exception {
-        doTest(CacheMode.REPLICATED, CacheAtomicityMode.TRANSACTIONAL, CacheMemoryMode.OFFHEAP_TIERED);
+        doTest(CacheMode.REPLICATED, CacheAtomicityMode.TRANSACTIONAL);
     }
 
     /**
      * @param cacheMode Cache mode.
      * @param atomicityMode Atomicity mode.
-     * @param memMode Memory mode.
      * @throws Exception If failed.
      */
-    private void doTest(CacheMode cacheMode, CacheAtomicityMode atomicityMode, CacheMemoryMode memMode)
+    private void doTest(CacheMode cacheMode, CacheAtomicityMode atomicityMode)
         throws Exception {
         Ignite ignite = grid("client");
 
@@ -164,7 +134,6 @@ public class CachePutEventListenerErrorSelfTest extends GridCommonAbstractTest {
             cfg.setName("cache");
             cfg.setCacheMode(cacheMode);
             cfg.setAtomicityMode(atomicityMode);
-            cfg.setMemoryMode(memMode);
 
             IgniteCache<Integer, Integer> cache = ignite.createCache(cfg);